You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2017/04/17 20:54:40 UTC

[01/50] [abbrv] hbase git commit: HBASE-17859 ByteBufferUtils#compareTo is wrong

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 1c4d9c896 -> ecdfb8232


HBASE-17859 ByteBufferUtils#compareTo is wrong


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

Branch: refs/heads/hbase-12439
Commit: 73e1bcd33515061be2dc2e51e6ad19d9798a8ef6
Parents: 9facfa5
Author: CHIA-PING TSAI <ch...@gmail.com>
Authored: Fri Mar 31 19:45:10 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Sat Apr 1 13:42:36 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/ByteBufferUtils.java      |  9 +++--
 .../hadoop/hbase/util/TestByteBufferUtils.java  | 39 ++++++++++++++++++++
 2 files changed, 45 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/73e1bcd3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 760afd4..4bed97c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -16,6 +16,7 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -49,8 +50,10 @@ public final class ByteBufferUtils {
   public final static int VALUE_MASK = 0x7f;
   public final static int NEXT_BIT_SHIFT = 7;
   public final static int NEXT_BIT_MASK = 1 << 7;
-  private static final boolean UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
-  private static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
+  @VisibleForTesting
+  static boolean UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
+  @VisibleForTesting
+  static boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
 
   private ByteBufferUtils() {
   }
@@ -668,7 +671,7 @@ public final class ByteBufferUtils {
     int end2 = o2 + l2;
     for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
       int a = buf1[i] & 0xFF;
-      int b = buf2.get(i) & 0xFF;
+      int b = buf2.get(j) & 0xFF;
       if (a != b) {
         return a - b;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e1bcd3/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
index b78574a..053fb24 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
@@ -28,8 +28,10 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -38,15 +40,44 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.io.WritableUtils;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 @Category({MiscTests.class, SmallTests.class})
+@RunWith(Parameterized.class)
 public class TestByteBufferUtils {
 
   private byte[] array;
 
+  @AfterClass
+  public static void afterClass() throws Exception {
+    ByteBufferUtils.UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
+    ByteBufferUtils.UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> parameters() {
+    List<Object[]> paramList = new ArrayList<>(2);
+    {
+      paramList.add(new Object[] { false });
+      paramList.add(new Object[] { true });
+    }
+    return paramList;
+  }
+
+  public TestByteBufferUtils(boolean useUnsafeIfPossible) {
+    if (useUnsafeIfPossible) {
+      ByteBufferUtils.UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
+      ByteBufferUtils.UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
+    } else {
+      ByteBufferUtils.UNSAFE_AVAIL = false;
+      ByteBufferUtils.UNSAFE_UNALIGNED = false;
+    }
+  }
   /**
    * Create an array with sample data.
    */
@@ -412,6 +443,14 @@ public class TestByteBufferUtils {
     assertTrue(result > 0);
     result = ByteBufferUtils.compareTo(bb3, 0, bb3.remaining(), b3, 0, b3.length);
     assertTrue(result < 0);
+
+    byte[] b4 = Bytes.toBytes("123");
+    ByteBuffer bb4 = ByteBuffer.allocate(10 + b4.length);
+    for (int i = 10; i < (bb4.capacity()); ++i) {
+      bb4.put(i, b4[i - 10]);
+    }
+    result = ByteBufferUtils.compareTo(b4, 0, b4.length, bb4, 10, b4.length);
+    assertEquals(0, result);
   }
 
   @Test


[16/50] [abbrv] hbase git commit: HBASE-17854 Use StealJobQueue in HFileCleaner after HBASE-17215

Posted by sy...@apache.org.
HBASE-17854 Use StealJobQueue in HFileCleaner after HBASE-17215


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

Branch: refs/heads/hbase-12439
Commit: cbcbcf4dcd3401327cc36173f3ca8e5362da1e0c
Parents: a66d491
Author: Yu Li <li...@apache.org>
Authored: Wed Apr 5 17:53:21 2017 +0800
Committer: Yu Li <li...@apache.org>
Committed: Wed Apr 5 17:53:21 2017 +0800

----------------------------------------------------------------------
 .../hbase/master/cleaner/HFileCleaner.java      | 98 +++++++++++++-------
 .../apache/hadoop/hbase/util/StealJobQueue.java | 22 +++++
 .../hbase/master/cleaner/TestHFileCleaner.java  | 28 +++---
 .../hadoop/hbase/util/TestStealJobQueue.java    |  2 +-
 4 files changed, 102 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cbcbcf4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index 3a68252..8b3515a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -35,6 +34,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.StealJobQueue;
 
 import com.google.common.annotations.VisibleForTesting;
 /**
@@ -57,23 +57,23 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       "hbase.regionserver.thread.hfilecleaner.throttle";
   public final static int DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD = 64 * 1024 * 1024;// 64M
 
-  // Configuration key for large queue size
-  public final static String LARGE_HFILE_DELETE_QUEUE_SIZE =
+  // Configuration key for large queue initial size
+  public final static String LARGE_HFILE_QUEUE_INIT_SIZE =
       "hbase.regionserver.hfilecleaner.large.queue.size";
-  public final static int DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE = 1048576;
+  public final static int DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE = 10240;
 
-  // Configuration key for small queue size
-  public final static String SMALL_HFILE_DELETE_QUEUE_SIZE =
+  // Configuration key for small queue initial size
+  public final static String SMALL_HFILE_QUEUE_INIT_SIZE =
       "hbase.regionserver.hfilecleaner.small.queue.size";
-  public final static int DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE = 1048576;
+  public final static int DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE = 10240;
 
   private static final Log LOG = LogFactory.getLog(HFileCleaner.class);
 
-  BlockingQueue<HFileDeleteTask> largeFileQueue;
+  StealJobQueue<HFileDeleteTask> largeFileQueue;
   BlockingQueue<HFileDeleteTask> smallFileQueue;
   private int throttlePoint;
-  private int largeQueueSize;
-  private int smallQueueSize;
+  private int largeQueueInitSize;
+  private int smallQueueInitSize;
   private List<Thread> threads = new ArrayList<Thread>();
   private boolean running;
 
@@ -94,12 +94,12 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       directory, MASTER_HFILE_CLEANER_PLUGINS, params);
     throttlePoint =
         conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
-    largeQueueSize =
-        conf.getInt(LARGE_HFILE_DELETE_QUEUE_SIZE, DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE);
-    smallQueueSize =
-        conf.getInt(SMALL_HFILE_DELETE_QUEUE_SIZE, DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE);
-    largeFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(largeQueueSize);
-    smallFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(smallQueueSize);
+    largeQueueInitSize =
+        conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
+    smallQueueInitSize =
+        conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
+    largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
+    smallFileQueue = largeFileQueue.getStealFromQueue();
     startHFileDeleteThreads();
   }
 
@@ -152,6 +152,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   private boolean dispatch(HFileDeleteTask task) {
     if (task.fileLength >= this.throttlePoint) {
       if (!this.largeFileQueue.offer(task)) {
+        // should never arrive here as long as we use PriorityQueue
         if (LOG.isTraceEnabled()) {
           LOG.trace("Large file deletion queue is full");
         }
@@ -159,6 +160,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       }
     } else {
       if (!this.smallFileQueue.offer(task)) {
+        // should never arrive here as long as we use PriorityQueue
         if (LOG.isTraceEnabled()) {
           LOG.trace("Small file deletion queue is full");
         }
@@ -232,7 +234,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
           }
           task.setResult(succeed);
           if (succeed) {
-            countDeletedFiles(queue == largeFileQueue);
+            countDeletedFiles(task.fileLength >= throttlePoint, queue == largeFileQueue);
           }
         }
       }
@@ -244,8 +246,8 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   }
 
   // Currently only for testing purpose
-  private void countDeletedFiles(boolean isLarge) {
-    if (isLarge) {
+  private void countDeletedFiles(boolean isLargeFile, boolean fromLargeQueue) {
+    if (isLargeFile) {
       if (deletedLargeFiles == Long.MAX_VALUE) {
         LOG.info("Deleted more than Long.MAX_VALUE large files, reset counter to 0");
         deletedLargeFiles = 0L;
@@ -256,6 +258,9 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
         LOG.info("Deleted more than Long.MAX_VALUE small files, reset counter to 0");
         deletedSmallFiles = 0L;
       }
+      if (fromLargeQueue && LOG.isTraceEnabled()) {
+        LOG.trace("Stolen a small file deletion task in large file thread");
+      }
       deletedSmallFiles++;
     }
   }
@@ -273,7 +278,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     }
   }
 
-  static class HFileDeleteTask {
+  static class HFileDeleteTask implements Comparable<HFileDeleteTask> {
     private static final long MAX_WAIT = 60 * 1000L;
     private static final long WAIT_UNIT = 1000L;
 
@@ -315,6 +320,31 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
       }
       return this.result;
     }
+
+    @Override
+    public int compareTo(HFileDeleteTask o) {
+      long sub = this.fileLength - o.fileLength;
+      // smaller value with higher priority in PriorityQueue, and we intent to delete the larger
+      // file first.
+      return (sub > 0) ? -1 : (sub < 0 ? 1 : 0);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || !(o instanceof HFileDeleteTask)) {
+        return false;
+      }
+      HFileDeleteTask otherTask = (HFileDeleteTask) o;
+      return this.filePath.equals(otherTask.filePath) && (this.fileLength == otherTask.fileLength);
+    }
+
+    @Override
+    public int hashCode() {
+      return filePath.hashCode();
+    }
   }
 
   @VisibleForTesting
@@ -333,13 +363,13 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   }
 
   @VisibleForTesting
-  public long getLargeQueueSize() {
-    return largeQueueSize;
+  public long getLargeQueueInitSize() {
+    return largeQueueInitSize;
   }
 
   @VisibleForTesting
-  public long getSmallQueueSize() {
-    return smallQueueSize;
+  public long getSmallQueueInitSize() {
+    return smallQueueInitSize;
   }
 
   @VisibleForTesting
@@ -351,15 +381,15 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
   public void onConfigurationChange(Configuration conf) {
     StringBuilder builder = new StringBuilder();
     builder.append("Updating configuration for HFileCleaner, previous throttle point: ")
-        .append(throttlePoint).append(", largeQueueSize: ").append(largeQueueSize)
-        .append(", smallQueueSize: ").append(smallQueueSize);
+        .append(throttlePoint).append(", largeQueueInitSize: ").append(largeQueueInitSize)
+        .append(", smallQueueInitSize: ").append(smallQueueInitSize);
     stopHFileDeleteThreads();
     this.throttlePoint =
         conf.getInt(HFILE_DELETE_THROTTLE_THRESHOLD, DEFAULT_HFILE_DELETE_THROTTLE_THRESHOLD);
-    this.largeQueueSize =
-        conf.getInt(LARGE_HFILE_DELETE_QUEUE_SIZE, DEFAULT_LARGE_HFILE_DELETE_QUEUE_SIZE);
-    this.smallQueueSize =
-        conf.getInt(SMALL_HFILE_DELETE_QUEUE_SIZE, DEFAULT_SMALL_HFILE_DELETE_QUEUE_SIZE);
+    this.largeQueueInitSize =
+        conf.getInt(LARGE_HFILE_QUEUE_INIT_SIZE, DEFAULT_LARGE_HFILE_QUEUE_INIT_SIZE);
+    this.smallQueueInitSize =
+        conf.getInt(SMALL_HFILE_QUEUE_INIT_SIZE, DEFAULT_SMALL_HFILE_QUEUE_INIT_SIZE);
     // record the left over tasks
     List<HFileDeleteTask> leftOverTasks = new ArrayList<>();
     for (HFileDeleteTask task : largeFileQueue) {
@@ -368,11 +398,11 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
     for (HFileDeleteTask task : smallFileQueue) {
       leftOverTasks.add(task);
     }
-    largeFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(largeQueueSize);
-    smallFileQueue = new LinkedBlockingQueue<HFileCleaner.HFileDeleteTask>(smallQueueSize);
+    largeFileQueue = new StealJobQueue<>(largeQueueInitSize, smallQueueInitSize);
+    smallFileQueue = largeFileQueue.getStealFromQueue();
     threads.clear();
-    builder.append("; new throttle point: ").append(throttlePoint).append(", largeQueueSize: ")
-        .append(largeQueueSize).append(", smallQueueSize: ").append(smallQueueSize);
+    builder.append("; new throttle point: ").append(throttlePoint).append(", largeQueueInitSize: ")
+        .append(largeQueueInitSize).append(", smallQueueInitSize: ").append(smallQueueInitSize);
     LOG.debug(builder.toString());
     startHFileDeleteThreads();
     // re-dispatch the left over tasks

http://git-wip-us.apache.org/repos/asf/hbase/blob/cbcbcf4d/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
index 74f0747..5e7e232 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/StealJobQueue.java
@@ -48,6 +48,24 @@ public class StealJobQueue<T> extends PriorityBlockingQueue<T> {
 
   public StealJobQueue() {
     this.stealFromQueue = new PriorityBlockingQueue<T>() {
+
+      @Override
+      public boolean offer(T t) {
+        lock.lock();
+        try {
+          notEmpty.signal();
+          return super.offer(t);
+        } finally {
+          lock.unlock();
+        }
+      }
+    };
+  }
+
+  public StealJobQueue(int initCapacity, int stealFromQueueInitCapacity) {
+    super(initCapacity);
+    this.stealFromQueue = new PriorityBlockingQueue<T>(stealFromQueueInitCapacity) {
+
       @Override
       public boolean offer(T t) {
         lock.lock();
@@ -61,6 +79,10 @@ public class StealJobQueue<T> extends PriorityBlockingQueue<T> {
     };
   }
 
+  /**
+   * Get a queue whose job might be stolen by the consumer of this original queue
+   * @return the queue whose job could be stolen
+   */
   public BlockingQueue<T> getStealFromQueue() {
     return stealFromQueue;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cbcbcf4d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 8e8a4dd..5f05488 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -322,9 +322,9 @@ public class TestHFileCleaner {
   public void testOnConfigurationChange() throws Exception {
     // constants
     final int ORIGINAL_THROTTLE_POINT = 512 * 1024;
-    final int ORIGINAL_QUEUE_SIZE = 512;
+    final int ORIGINAL_QUEUE_INIT_SIZE = 512;
     final int UPDATE_THROTTLE_POINT = 1024;// small enough to change large/small check
-    final int UPDATE_QUEUE_SIZE = 1024;
+    final int UPDATE_QUEUE_INIT_SIZE = 1024;
     final int LARGE_FILE_NUM = 5;
     final int SMALL_FILE_NUM = 20;
 
@@ -332,8 +332,8 @@ public class TestHFileCleaner {
     // no cleaner policies = delete all files
     conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, "");
     conf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, ORIGINAL_THROTTLE_POINT);
-    conf.setInt(HFileCleaner.LARGE_HFILE_DELETE_QUEUE_SIZE, ORIGINAL_QUEUE_SIZE);
-    conf.setInt(HFileCleaner.SMALL_HFILE_DELETE_QUEUE_SIZE, ORIGINAL_QUEUE_SIZE);
+    conf.setInt(HFileCleaner.LARGE_HFILE_QUEUE_INIT_SIZE, ORIGINAL_QUEUE_INIT_SIZE);
+    conf.setInt(HFileCleaner.SMALL_HFILE_QUEUE_INIT_SIZE, ORIGINAL_QUEUE_INIT_SIZE);
     Server server = new DummyServer();
     Path archivedHfileDir =
         new Path(UTIL.getDataTestDirOnTestFS(), HConstants.HFILE_ARCHIVE_DIRECTORY);
@@ -342,8 +342,8 @@ public class TestHFileCleaner {
     FileSystem fs = UTIL.getDFSCluster().getFileSystem();
     final HFileCleaner cleaner = new HFileCleaner(1000, server, conf, fs, archivedHfileDir);
     Assert.assertEquals(ORIGINAL_THROTTLE_POINT, cleaner.getThrottlePoint());
-    Assert.assertEquals(ORIGINAL_QUEUE_SIZE, cleaner.getLargeQueueSize());
-    Assert.assertEquals(ORIGINAL_QUEUE_SIZE, cleaner.getSmallQueueSize());
+    Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
+    Assert.assertEquals(ORIGINAL_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
 
     // clean up archive directory and create files for testing
     fs.delete(archivedHfileDir, true);
@@ -359,22 +359,24 @@ public class TestHFileCleaner {
     };
     t.setDaemon(true);
     t.start();
-    // let the cleaner run for some while
-    Thread.sleep(20);
+    // wait until file clean started
+    while (cleaner.getNumOfDeletedSmallFiles() == 0) {
+      Thread.yield();
+    }
 
     // trigger configuration change
     Configuration newConf = new Configuration(conf);
     newConf.setInt(HFileCleaner.HFILE_DELETE_THROTTLE_THRESHOLD, UPDATE_THROTTLE_POINT);
-    newConf.setInt(HFileCleaner.LARGE_HFILE_DELETE_QUEUE_SIZE, UPDATE_QUEUE_SIZE);
-    newConf.setInt(HFileCleaner.SMALL_HFILE_DELETE_QUEUE_SIZE, UPDATE_QUEUE_SIZE);
-    cleaner.onConfigurationChange(newConf);
+    newConf.setInt(HFileCleaner.LARGE_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
+    newConf.setInt(HFileCleaner.SMALL_HFILE_QUEUE_INIT_SIZE, UPDATE_QUEUE_INIT_SIZE);
     LOG.debug("File deleted from large queue: " + cleaner.getNumOfDeletedLargeFiles()
         + "; from small queue: " + cleaner.getNumOfDeletedSmallFiles());
+    cleaner.onConfigurationChange(newConf);
 
     // check values after change
     Assert.assertEquals(UPDATE_THROTTLE_POINT, cleaner.getThrottlePoint());
-    Assert.assertEquals(UPDATE_QUEUE_SIZE, cleaner.getLargeQueueSize());
-    Assert.assertEquals(UPDATE_QUEUE_SIZE, cleaner.getSmallQueueSize());
+    Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getLargeQueueInitSize());
+    Assert.assertEquals(UPDATE_QUEUE_INIT_SIZE, cleaner.getSmallQueueInitSize());
     Assert.assertEquals(2, cleaner.getCleanerThreads().size());
 
     // wait until clean done and check

http://git-wip-us.apache.org/repos/asf/hbase/blob/cbcbcf4d/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
index b35f6f4..54fdaca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestStealJobQueue.java
@@ -38,7 +38,7 @@ import static org.junit.Assert.*;
 public class TestStealJobQueue {
 
   StealJobQueue<Integer> stealJobQueue;
-  BlockingQueue stealFromQueue;
+  BlockingQueue<Integer> stealFromQueue;
 
   @Before
   public void setup() {


[47/50] [abbrv] hbase git commit: HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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 d56d6ec..095f4bd 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
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterAllFilter;
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.NullComparator;
@@ -4931,6 +4932,7 @@ public class TestHRegion {
       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
       throws IOException {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
     return initHRegion(tableName, startKey, stopKey, isReadOnly,

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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 0054642..6eed7df 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
@@ -153,7 +153,7 @@ public class TestHRegionReplayEvents {
     }
 
     time = System.currentTimeMillis();
-
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     primaryHri = new HRegionInfo(htd.getTableName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
       false, time, 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index 37a7664..1768801 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -48,30 +48,30 @@ import static org.junit.Assert.assertTrue;
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestMemStoreChunkPool {
   private final static Configuration conf = new Configuration();
-  private static MemStoreChunkPool chunkPool;
+  private static ChunkCreator chunkCreator;
   private static boolean chunkPoolDisabledBeforeTest;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    chunkPoolDisabledBeforeTest = MemStoreChunkPool.chunkPoolDisabled;
-    MemStoreChunkPool.chunkPoolDisabled = false;
+    chunkPoolDisabledBeforeTest = ChunkCreator.chunkPoolDisabled;
+    ChunkCreator.chunkPoolDisabled = false;
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
-        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
-    assertTrue(chunkPool != null);
+    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
+      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+    assertTrue(chunkCreator != null);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    MemStoreChunkPool.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
+    ChunkCreator.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
   }
 
   @Before
   public void tearDown() throws Exception {
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
   }
 
   @Test
@@ -90,7 +90,7 @@ public class TestMemStoreChunkPool {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        expectedOff = 0;
+        expectedOff = 8;
         lastBuffer = newKv.getBuffer();
       }
       assertEquals(expectedOff, newKv.getOffset());
@@ -100,14 +100,14 @@ public class TestMemStoreChunkPool {
     }
     // chunks will be put back to pool after close
     mslab.close();
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
     // reconstruct mslab
     mslab = new MemStoreLABImpl(conf);
     // chunk should be got from the pool, so we can reuse it.
     KeyValue kv = new KeyValue(rk, cf, q, new byte[10]);
     mslab.copyCellInto(kv);
-    assertEquals(chunkCount - 1, chunkPool.getPoolSize());
+    assertEquals(chunkCount - 1, chunkCreator.getPoolSize());
   }
 
   @Test
@@ -143,7 +143,7 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -189,16 +189,16 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
     snapshot = memstore.snapshot();
@@ -218,20 +218,20 @@ public class TestMemStoreChunkPool {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   @Test
   public void testPutbackChunksMultiThreaded() throws Exception {
-    MemStoreChunkPool oldPool = MemStoreChunkPool.GLOBAL_INSTANCE;
     final int maxCount = 10;
     final int initialCount = 5;
-    final int chunkSize = 30;
+    final int chunkSize = 40;
     final int valSize = 7;
-    MemStoreChunkPool pool = new MemStoreChunkPool(chunkSize, maxCount, initialCount, 1, false);
-    assertEquals(initialCount, pool.getPoolSize());
-    assertEquals(maxCount, pool.getMaxCount());
-    MemStoreChunkPool.GLOBAL_INSTANCE = pool;// Replace the global ref with the new one we created.
+    ChunkCreator oldCreator = ChunkCreator.getInstance();
+    ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, null);
+    assertEquals(initialCount, newCreator.getPoolSize());
+    assertEquals(maxCount, newCreator.getMaxCount());
+    ChunkCreator.INSTANCE = newCreator;// Replace the global ref with the new one we created.
                                              // Used it for the testing. Later in finally we put
                                              // back the original
     final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
@@ -258,9 +258,9 @@ public class TestMemStoreChunkPool {
       t1.join();
       t2.join();
       t3.join();
-      assertTrue(pool.getPoolSize() <= maxCount);
+      assertTrue(newCreator.getPoolSize() <= maxCount);
     } finally {
-      MemStoreChunkPool.GLOBAL_INSTANCE = oldPool;
+      ChunkCreator.INSTANCE = oldCreator;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
index 141b802..6696e43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
@@ -63,8 +63,8 @@ public class TestMemStoreLAB {
   public static void setUpBeforeClass() throws Exception {
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT,
-        MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, globalMemStoreLimit,
+      0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
   }
 
   /**
@@ -76,6 +76,7 @@ public class TestMemStoreLAB {
     MemStoreLAB mslab = new MemStoreLABImpl();
     int expectedOff = 0;
     ByteBuffer lastBuffer = null;
+    long lastChunkId = -1;
     // 100K iterations by 0-1K alloc -> 50MB expected
     // should be reasonable for unit test and also cover wraparound
     // behavior
@@ -85,8 +86,13 @@ public class TestMemStoreLAB {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        expectedOff = 0;
+        // since we add the chunkID at the 0th offset of the chunk and the
+        // chunkid is a long we need to account for those 8 bytes
+        expectedOff = Bytes.SIZEOF_LONG;
         lastBuffer = newKv.getBuffer();
+        long chunkId = newKv.getBuffer().getLong(0);
+        assertTrue("chunkid should be different", chunkId != lastChunkId);
+        lastChunkId = chunkId;
       }
       assertEquals(expectedOff, newKv.getOffset());
       assertTrue("Allocation overruns buffer",
@@ -136,23 +142,21 @@ public class TestMemStoreLAB {
       };
       ctx.addThread(t);
     }
-    
+
     ctx.startThreads();
     while (totalAllocated.get() < 50*1024*1024 && ctx.shouldRun()) {
       Thread.sleep(10);
     }
     ctx.stop();
-    
     // Partition the allocations by the actual byte[] they point into,
     // make sure offsets are unique for each chunk
     Map<ByteBuffer, Map<Integer, AllocRecord>> mapsByChunk =
       Maps.newHashMap();
-    
+
     int sizeCounted = 0;
     for (AllocRecord rec : Iterables.concat(allocations)) {
       sizeCounted += rec.size;
       if (rec.size == 0) continue;
-      
       Map<Integer, AllocRecord> mapForThisByteArray =
         mapsByChunk.get(rec.alloc);
       if (mapForThisByteArray == null) {
@@ -167,7 +171,9 @@ public class TestMemStoreLAB {
     
     // Now check each byte array to make sure allocations don't overlap
     for (Map<Integer, AllocRecord> allocsInChunk : mapsByChunk.values()) {
-      int expectedOff = 0;
+      // since we add the chunkID at the 0th offset of the chunk and the
+      // chunkid is a long we need to account for those 8 bytes
+      int expectedOff = Bytes.SIZEOF_LONG;
       for (AllocRecord alloc : allocsInChunk.values()) {
         assertEquals(expectedOff, alloc.offset);
         assertTrue("Allocation overruns buffer",
@@ -175,7 +181,6 @@ public class TestMemStoreLAB {
         expectedOff += alloc.size;
       }
     }
-
   }
 
   /**
@@ -194,7 +199,7 @@ public class TestMemStoreLAB {
     // set chunk size to default max alloc size, so we could easily trigger chunk retirement
     conf.setLong(MemStoreLABImpl.CHUNK_SIZE_KEY, MemStoreLABImpl.MAX_ALLOC_DEFAULT);
     // reconstruct mslab
-    MemStoreChunkPool.clearDisableFlag();
+    ChunkCreator.clearDisableFlag();
     mslab = new MemStoreLABImpl(conf);
     // launch multiple threads to trigger frequent chunk retirement
     List<Thread> threads = new ArrayList<>();
@@ -223,6 +228,8 @@ public class TestMemStoreLAB {
     }
     // close the mslab
     mslab.close();
+    // none of the chunkIds would have been returned back
+    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() != 0);
     // make sure all chunks reclaimed or removed from chunk queue
     int queueLength = mslab.getPooledChunks().size();
     assertTrue("All chunks in chunk queue should be reclaimed or removed"

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
new file mode 100644
index 0000000..f38a75e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.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.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.lang.management.ManagementFactory;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({RegionServerTests.class, SmallTests.class})
+public class TestMemstoreLABWithoutPool {
+  private final static Configuration conf = new Configuration();
+
+  private static final byte[] rk = Bytes.toBytes("r1");
+  private static final byte[] cf = Bytes.toBytes("f");
+  private static final byte[] q = Bytes.toBytes("q");
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
+        .getMax() * 0.8);
+    // disable pool
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT + Bytes.SIZEOF_LONG, false, globalMemStoreLimit,
+      0.0f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+  }
+
+  /**
+   * Test a bunch of random allocations
+   */
+  @Test
+  public void testLABRandomAllocation() {
+    Random rand = new Random();
+    MemStoreLAB mslab = new MemStoreLABImpl();
+    int expectedOff = 0;
+    ByteBuffer lastBuffer = null;
+    long lastChunkId = -1;
+    // 100K iterations by 0-1K alloc -> 50MB expected
+    // should be reasonable for unit test and also cover wraparound
+    // behavior
+    for (int i = 0; i < 100000; i++) {
+      int valSize = rand.nextInt(1000);
+      KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
+      int size = KeyValueUtil.length(kv);
+      ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
+      if (newKv.getBuffer() != lastBuffer) {
+        // since we add the chunkID at the 0th offset of the chunk and the
+        // chunkid is a long we need to account for those 8 bytes
+        expectedOff = Bytes.SIZEOF_LONG;
+        lastBuffer = newKv.getBuffer();
+        long chunkId = newKv.getBuffer().getLong(0);
+        assertTrue("chunkid should be different", chunkId != lastChunkId);
+        lastChunkId = chunkId;
+      }
+      assertEquals(expectedOff, newKv.getOffset());
+      assertTrue("Allocation overruns buffer",
+          newKv.getOffset() + size <= newKv.getBuffer().capacity());
+      expectedOff += size;
+    }
+  }
+
+  /**
+   * Test frequent chunk retirement with chunk pool triggered by lots of threads, making sure
+   * there's no memory leak (HBASE-16195)
+   * @throws Exception if any error occurred
+   */
+  @Test
+  public void testLABChunkQueueWithMultipleMSLABs() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    MemStoreLABImpl[] mslab = new MemStoreLABImpl[10];
+    for (int i = 0; i < 10; i++) {
+      mslab[i] = new MemStoreLABImpl(conf);
+    }
+    // launch multiple threads to trigger frequent chunk retirement
+    List<Thread> threads = new ArrayList<>();
+    // create smaller sized kvs
+    final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
+        new byte[0]);
+    for (int i = 0; i < 10; i++) {
+      for (int j = 0; j < 10; j++) {
+        threads.add(getChunkQueueTestThread(mslab[i], "testLABChunkQueue-" + j, kv));
+      }
+    }
+    for (Thread thread : threads) {
+      thread.start();
+    }
+    // let it run for some time
+    Thread.sleep(3000);
+    for (Thread thread : threads) {
+      thread.interrupt();
+    }
+    boolean threadsRunning = true;
+    boolean alive = false;
+    while (threadsRunning) {
+      alive = false;
+      for (Thread thread : threads) {
+        if (thread.isAlive()) {
+          alive = true;
+          break;
+        }
+      }
+      if (!alive) {
+        threadsRunning = false;
+      }
+    }
+    // close the mslab
+    for (int i = 0; i < 10; i++) {
+      mslab[i].close();
+    }
+    // all of the chunkIds would have been returned back
+    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() == 0);
+  }
+
+  private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,
+      Cell cellToCopyInto) {
+    Thread thread = new Thread() {
+      boolean stopped = false;
+
+      @Override
+      public void run() {
+        while (!stopped) {
+          // keep triggering chunk retirement
+          mslab.copyCellInto(cellToCopyInto);
+        }
+      }
+
+      @Override
+      public void interrupt() {
+        this.stopped = true;
+      }
+    };
+    thread.setName(threadName);
+    thread.setDaemon(true);
+    return thread;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 4315bd4..7160e5e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -108,6 +108,7 @@ public class TestRecoveredEdits {
       }
     };
     Path hbaseRootDir = TEST_UTIL.getDataTestDir();
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
     Path tableDir = FSUtils.getTableDir(hbaseRootDir, htd.getTableName());
     HRegionFileSystem hrfs =

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index 5d11c0e..ad56081 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -83,6 +83,7 @@ public class TestRegionIncrement {
   private HRegion getRegion(final Configuration conf, final String tableName) throws IOException {
     WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
       TEST_UTIL.getDataTestDir().toString(), conf);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, conf,
       false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 0d339b1..5355c77 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -178,6 +178,7 @@ public class TestStore {
     } else {
       htd.addFamily(hcd);
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 3cdb227..99dd00d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -111,6 +111,7 @@ public class TestStoreFileRefresherChore {
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region =
         new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
             conf, htd, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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 4f247b0..51260a6 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
@@ -588,6 +588,7 @@ public class TestWALLockup {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index 994779f..e63bad9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -98,6 +98,7 @@ public class TestWALMonotonicallyIncreasingSeqId {
     FSUtils.setRootDir(walConf, tableDir);
     this.walConf = walConf;
     wals = new WALFactory(walConf, null, "log_" + replicaId);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region = new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(),
         info.getTable().getNamespace()), conf, htd, null);
     region.initialize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index f976b49..057b9bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -37,7 +37,9 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -288,6 +290,7 @@ public class TestDurability {
           throw new IOException("Failed delete of " + path);
         }
       }
+      ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
       return HRegion.createHRegion(info, path, CONF, htd, log);
     }
 


[06/50] [abbrv] hbase git commit: HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistryLite.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistryLite.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistryLite.java
index 878e46a..9bf452a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistryLite.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistryLite.java
@@ -105,9 +105,9 @@ public class ExtensionRegistryLite {
 
   /**
    * Construct a new, empty instance.
-   * 
-   * <p>
-   * This may be an {@code ExtensionRegistry} if the full (non-Lite) proto libraries are available.
+   *
+   * <p>This may be an {@code ExtensionRegistry} if the full (non-Lite) proto libraries are
+   * available.
    */
   public static ExtensionRegistryLite newInstance() {
     return ExtensionRegistryFactory.create();
@@ -121,6 +121,7 @@ public class ExtensionRegistryLite {
     return ExtensionRegistryFactory.createEmpty();
   }
 
+
   /** Returns an unmodifiable view of the registry. */
   public ExtensionRegistryLite getUnmodifiable() {
     return new ExtensionRegistryLite(this);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Field.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Field.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Field.java
index 15951b3..d33fd75 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Field.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Field.java
@@ -709,7 +709,7 @@ public  final class Field extends
    * The field type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+   * <code>.google.protobuf.Field.Kind kind = 1;</code>
    */
   public int getKindValue() {
     return kind_;
@@ -719,7 +719,7 @@ public  final class Field extends
    * The field type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+   * <code>.google.protobuf.Field.Kind kind = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind getKind() {
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind.valueOf(kind_);
@@ -733,7 +733,7 @@ public  final class Field extends
    * The field cardinality.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+   * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
    */
   public int getCardinalityValue() {
     return cardinality_;
@@ -743,7 +743,7 @@ public  final class Field extends
    * The field cardinality.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+   * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality getCardinality() {
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality.valueOf(cardinality_);
@@ -757,7 +757,7 @@ public  final class Field extends
    * The field number.
    * </pre>
    *
-   * <code>optional int32 number = 3;</code>
+   * <code>int32 number = 3;</code>
    */
   public int getNumber() {
     return number_;
@@ -770,7 +770,7 @@ public  final class Field extends
    * The field name.
    * </pre>
    *
-   * <code>optional string name = 4;</code>
+   * <code>string name = 4;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -789,7 +789,7 @@ public  final class Field extends
    * The field name.
    * </pre>
    *
-   * <code>optional string name = 4;</code>
+   * <code>string name = 4;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -813,7 +813,7 @@ public  final class Field extends
    * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
    * </pre>
    *
-   * <code>optional string type_url = 6;</code>
+   * <code>string type_url = 6;</code>
    */
   public java.lang.String getTypeUrl() {
     java.lang.Object ref = typeUrl_;
@@ -833,7 +833,7 @@ public  final class Field extends
    * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
    * </pre>
    *
-   * <code>optional string type_url = 6;</code>
+   * <code>string type_url = 6;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getTypeUrlBytes() {
@@ -857,7 +857,7 @@ public  final class Field extends
    * types. The first type has index 1; zero means the type is not in the list.
    * </pre>
    *
-   * <code>optional int32 oneof_index = 7;</code>
+   * <code>int32 oneof_index = 7;</code>
    */
   public int getOneofIndex() {
     return oneofIndex_;
@@ -870,7 +870,7 @@ public  final class Field extends
    * Whether to use alternative packed wire representation.
    * </pre>
    *
-   * <code>optional bool packed = 8;</code>
+   * <code>bool packed = 8;</code>
    */
   public boolean getPacked() {
     return packed_;
@@ -938,7 +938,7 @@ public  final class Field extends
    * The field JSON name.
    * </pre>
    *
-   * <code>optional string json_name = 10;</code>
+   * <code>string json_name = 10;</code>
    */
   public java.lang.String getJsonName() {
     java.lang.Object ref = jsonName_;
@@ -957,7 +957,7 @@ public  final class Field extends
    * The field JSON name.
    * </pre>
    *
-   * <code>optional string json_name = 10;</code>
+   * <code>string json_name = 10;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getJsonNameBytes() {
@@ -980,7 +980,7 @@ public  final class Field extends
    * The string value of the default value of this field. Proto2 syntax only.
    * </pre>
    *
-   * <code>optional string default_value = 11;</code>
+   * <code>string default_value = 11;</code>
    */
   public java.lang.String getDefaultValue() {
     java.lang.Object ref = defaultValue_;
@@ -999,7 +999,7 @@ public  final class Field extends
    * The string value of the default value of this field. Proto2 syntax only.
    * </pre>
    *
-   * <code>optional string default_value = 11;</code>
+   * <code>string default_value = 11;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getDefaultValueBytes() {
@@ -1143,7 +1143,7 @@ public  final class Field extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + KIND_FIELD_NUMBER;
     hash = (53 * hash) + kind_;
     hash = (37 * hash) + CARDINALITY_FIELD_NUMBER;
@@ -1488,7 +1488,7 @@ public  final class Field extends
      * The field type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+     * <code>.google.protobuf.Field.Kind kind = 1;</code>
      */
     public int getKindValue() {
       return kind_;
@@ -1498,7 +1498,7 @@ public  final class Field extends
      * The field type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+     * <code>.google.protobuf.Field.Kind kind = 1;</code>
      */
     public Builder setKindValue(int value) {
       kind_ = value;
@@ -1510,7 +1510,7 @@ public  final class Field extends
      * The field type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+     * <code>.google.protobuf.Field.Kind kind = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind getKind() {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind.valueOf(kind_);
@@ -1521,7 +1521,7 @@ public  final class Field extends
      * The field type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+     * <code>.google.protobuf.Field.Kind kind = 1;</code>
      */
     public Builder setKind(org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind value) {
       if (value == null) {
@@ -1537,7 +1537,7 @@ public  final class Field extends
      * The field type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+     * <code>.google.protobuf.Field.Kind kind = 1;</code>
      */
     public Builder clearKind() {
       
@@ -1552,7 +1552,7 @@ public  final class Field extends
      * The field cardinality.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+     * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
      */
     public int getCardinalityValue() {
       return cardinality_;
@@ -1562,7 +1562,7 @@ public  final class Field extends
      * The field cardinality.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+     * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
      */
     public Builder setCardinalityValue(int value) {
       cardinality_ = value;
@@ -1574,7 +1574,7 @@ public  final class Field extends
      * The field cardinality.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+     * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality getCardinality() {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality.valueOf(cardinality_);
@@ -1585,7 +1585,7 @@ public  final class Field extends
      * The field cardinality.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+     * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
      */
     public Builder setCardinality(org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality value) {
       if (value == null) {
@@ -1601,7 +1601,7 @@ public  final class Field extends
      * The field cardinality.
      * </pre>
      *
-     * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+     * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
      */
     public Builder clearCardinality() {
       
@@ -1616,7 +1616,7 @@ public  final class Field extends
      * The field number.
      * </pre>
      *
-     * <code>optional int32 number = 3;</code>
+     * <code>int32 number = 3;</code>
      */
     public int getNumber() {
       return number_;
@@ -1626,7 +1626,7 @@ public  final class Field extends
      * The field number.
      * </pre>
      *
-     * <code>optional int32 number = 3;</code>
+     * <code>int32 number = 3;</code>
      */
     public Builder setNumber(int value) {
       
@@ -1639,7 +1639,7 @@ public  final class Field extends
      * The field number.
      * </pre>
      *
-     * <code>optional int32 number = 3;</code>
+     * <code>int32 number = 3;</code>
      */
     public Builder clearNumber() {
       
@@ -1654,7 +1654,7 @@ public  final class Field extends
      * The field name.
      * </pre>
      *
-     * <code>optional string name = 4;</code>
+     * <code>string name = 4;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -1673,7 +1673,7 @@ public  final class Field extends
      * The field name.
      * </pre>
      *
-     * <code>optional string name = 4;</code>
+     * <code>string name = 4;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -1693,7 +1693,7 @@ public  final class Field extends
      * The field name.
      * </pre>
      *
-     * <code>optional string name = 4;</code>
+     * <code>string name = 4;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -1710,7 +1710,7 @@ public  final class Field extends
      * The field name.
      * </pre>
      *
-     * <code>optional string name = 4;</code>
+     * <code>string name = 4;</code>
      */
     public Builder clearName() {
       
@@ -1723,7 +1723,7 @@ public  final class Field extends
      * The field name.
      * </pre>
      *
-     * <code>optional string name = 4;</code>
+     * <code>string name = 4;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -1744,7 +1744,7 @@ public  final class Field extends
      * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
      * </pre>
      *
-     * <code>optional string type_url = 6;</code>
+     * <code>string type_url = 6;</code>
      */
     public java.lang.String getTypeUrl() {
       java.lang.Object ref = typeUrl_;
@@ -1764,7 +1764,7 @@ public  final class Field extends
      * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
      * </pre>
      *
-     * <code>optional string type_url = 6;</code>
+     * <code>string type_url = 6;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getTypeUrlBytes() {
@@ -1785,7 +1785,7 @@ public  final class Field extends
      * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
      * </pre>
      *
-     * <code>optional string type_url = 6;</code>
+     * <code>string type_url = 6;</code>
      */
     public Builder setTypeUrl(
         java.lang.String value) {
@@ -1803,7 +1803,7 @@ public  final class Field extends
      * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
      * </pre>
      *
-     * <code>optional string type_url = 6;</code>
+     * <code>string type_url = 6;</code>
      */
     public Builder clearTypeUrl() {
       
@@ -1817,7 +1817,7 @@ public  final class Field extends
      * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
      * </pre>
      *
-     * <code>optional string type_url = 6;</code>
+     * <code>string type_url = 6;</code>
      */
     public Builder setTypeUrlBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -1838,7 +1838,7 @@ public  final class Field extends
      * types. The first type has index 1; zero means the type is not in the list.
      * </pre>
      *
-     * <code>optional int32 oneof_index = 7;</code>
+     * <code>int32 oneof_index = 7;</code>
      */
     public int getOneofIndex() {
       return oneofIndex_;
@@ -1849,7 +1849,7 @@ public  final class Field extends
      * types. The first type has index 1; zero means the type is not in the list.
      * </pre>
      *
-     * <code>optional int32 oneof_index = 7;</code>
+     * <code>int32 oneof_index = 7;</code>
      */
     public Builder setOneofIndex(int value) {
       
@@ -1863,7 +1863,7 @@ public  final class Field extends
      * types. The first type has index 1; zero means the type is not in the list.
      * </pre>
      *
-     * <code>optional int32 oneof_index = 7;</code>
+     * <code>int32 oneof_index = 7;</code>
      */
     public Builder clearOneofIndex() {
       
@@ -1878,7 +1878,7 @@ public  final class Field extends
      * Whether to use alternative packed wire representation.
      * </pre>
      *
-     * <code>optional bool packed = 8;</code>
+     * <code>bool packed = 8;</code>
      */
     public boolean getPacked() {
       return packed_;
@@ -1888,7 +1888,7 @@ public  final class Field extends
      * Whether to use alternative packed wire representation.
      * </pre>
      *
-     * <code>optional bool packed = 8;</code>
+     * <code>bool packed = 8;</code>
      */
     public Builder setPacked(boolean value) {
       
@@ -1901,7 +1901,7 @@ public  final class Field extends
      * Whether to use alternative packed wire representation.
      * </pre>
      *
-     * <code>optional bool packed = 8;</code>
+     * <code>bool packed = 8;</code>
      */
     public Builder clearPacked() {
       
@@ -2228,7 +2228,7 @@ public  final class Field extends
      * The field JSON name.
      * </pre>
      *
-     * <code>optional string json_name = 10;</code>
+     * <code>string json_name = 10;</code>
      */
     public java.lang.String getJsonName() {
       java.lang.Object ref = jsonName_;
@@ -2247,7 +2247,7 @@ public  final class Field extends
      * The field JSON name.
      * </pre>
      *
-     * <code>optional string json_name = 10;</code>
+     * <code>string json_name = 10;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getJsonNameBytes() {
@@ -2267,7 +2267,7 @@ public  final class Field extends
      * The field JSON name.
      * </pre>
      *
-     * <code>optional string json_name = 10;</code>
+     * <code>string json_name = 10;</code>
      */
     public Builder setJsonName(
         java.lang.String value) {
@@ -2284,7 +2284,7 @@ public  final class Field extends
      * The field JSON name.
      * </pre>
      *
-     * <code>optional string json_name = 10;</code>
+     * <code>string json_name = 10;</code>
      */
     public Builder clearJsonName() {
       
@@ -2297,7 +2297,7 @@ public  final class Field extends
      * The field JSON name.
      * </pre>
      *
-     * <code>optional string json_name = 10;</code>
+     * <code>string json_name = 10;</code>
      */
     public Builder setJsonNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -2317,7 +2317,7 @@ public  final class Field extends
      * The string value of the default value of this field. Proto2 syntax only.
      * </pre>
      *
-     * <code>optional string default_value = 11;</code>
+     * <code>string default_value = 11;</code>
      */
     public java.lang.String getDefaultValue() {
       java.lang.Object ref = defaultValue_;
@@ -2336,7 +2336,7 @@ public  final class Field extends
      * The string value of the default value of this field. Proto2 syntax only.
      * </pre>
      *
-     * <code>optional string default_value = 11;</code>
+     * <code>string default_value = 11;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getDefaultValueBytes() {
@@ -2356,7 +2356,7 @@ public  final class Field extends
      * The string value of the default value of this field. Proto2 syntax only.
      * </pre>
      *
-     * <code>optional string default_value = 11;</code>
+     * <code>string default_value = 11;</code>
      */
     public Builder setDefaultValue(
         java.lang.String value) {
@@ -2373,7 +2373,7 @@ public  final class Field extends
      * The string value of the default value of this field. Proto2 syntax only.
      * </pre>
      *
-     * <code>optional string default_value = 11;</code>
+     * <code>string default_value = 11;</code>
      */
     public Builder clearDefaultValue() {
       
@@ -2386,7 +2386,7 @@ public  final class Field extends
      * The string value of the default value of this field. Proto2 syntax only.
      * </pre>
      *
-     * <code>optional string default_value = 11;</code>
+     * <code>string default_value = 11;</code>
      */
     public Builder setDefaultValueBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMask.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMask.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMask.java
index 9c311e4..df42bca 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMask.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMask.java
@@ -336,7 +336,7 @@ public  final class FieldMask extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     if (getPathsCount() > 0) {
       hash = (37 * hash) + PATHS_FIELD_NUMBER;
       hash = (53 * hash) + getPathsList().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMaskProto.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMaskProto.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMaskProto.java
index 711e94f..4e978bf 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMaskProto.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldMaskProto.java
@@ -30,9 +30,10 @@ public final class FieldMaskProto {
     java.lang.String[] descriptorData = {
       "\n google/protobuf/field_mask.proto\022\017goog" +
       "le.protobuf\"\032\n\tFieldMask\022\r\n\005paths\030\001 \003(\tB" +
-      "N\n\023com.google.protobufB\016FieldMaskProtoP\001" +
-      "\242\002\003GPB\252\002\036Google.Protobuf.WellKnownTypesb" +
-      "\006proto3"
+      "\211\001\n\023com.google.protobufB\016FieldMaskProtoP" +
+      "\001Z9google.golang.org/genproto/protobuf/f" +
+      "ield_mask;field_mask\242\002\003GPB\252\002\036Google.Prot" +
+      "obuf.WellKnownTypesb\006proto3"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldOrBuilder.java
index 238e17a..1bd3ada 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldOrBuilder.java
@@ -12,7 +12,7 @@ public interface FieldOrBuilder extends
    * The field type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+   * <code>.google.protobuf.Field.Kind kind = 1;</code>
    */
   int getKindValue();
   /**
@@ -20,7 +20,7 @@ public interface FieldOrBuilder extends
    * The field type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Kind kind = 1;</code>
+   * <code>.google.protobuf.Field.Kind kind = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Kind getKind();
 
@@ -29,7 +29,7 @@ public interface FieldOrBuilder extends
    * The field cardinality.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+   * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
    */
   int getCardinalityValue();
   /**
@@ -37,7 +37,7 @@ public interface FieldOrBuilder extends
    * The field cardinality.
    * </pre>
    *
-   * <code>optional .google.protobuf.Field.Cardinality cardinality = 2;</code>
+   * <code>.google.protobuf.Field.Cardinality cardinality = 2;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Field.Cardinality getCardinality();
 
@@ -46,7 +46,7 @@ public interface FieldOrBuilder extends
    * The field number.
    * </pre>
    *
-   * <code>optional int32 number = 3;</code>
+   * <code>int32 number = 3;</code>
    */
   int getNumber();
 
@@ -55,7 +55,7 @@ public interface FieldOrBuilder extends
    * The field name.
    * </pre>
    *
-   * <code>optional string name = 4;</code>
+   * <code>string name = 4;</code>
    */
   java.lang.String getName();
   /**
@@ -63,7 +63,7 @@ public interface FieldOrBuilder extends
    * The field name.
    * </pre>
    *
-   * <code>optional string name = 4;</code>
+   * <code>string name = 4;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
@@ -74,7 +74,7 @@ public interface FieldOrBuilder extends
    * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
    * </pre>
    *
-   * <code>optional string type_url = 6;</code>
+   * <code>string type_url = 6;</code>
    */
   java.lang.String getTypeUrl();
   /**
@@ -83,7 +83,7 @@ public interface FieldOrBuilder extends
    * types. Example: `"type.googleapis.org.apache.hadoop.hbase.shaded.com.google.protobuf.Timestamp"`.
    * </pre>
    *
-   * <code>optional string type_url = 6;</code>
+   * <code>string type_url = 6;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getTypeUrlBytes();
@@ -94,7 +94,7 @@ public interface FieldOrBuilder extends
    * types. The first type has index 1; zero means the type is not in the list.
    * </pre>
    *
-   * <code>optional int32 oneof_index = 7;</code>
+   * <code>int32 oneof_index = 7;</code>
    */
   int getOneofIndex();
 
@@ -103,7 +103,7 @@ public interface FieldOrBuilder extends
    * Whether to use alternative packed wire representation.
    * </pre>
    *
-   * <code>optional bool packed = 8;</code>
+   * <code>bool packed = 8;</code>
    */
   boolean getPacked();
 
@@ -156,7 +156,7 @@ public interface FieldOrBuilder extends
    * The field JSON name.
    * </pre>
    *
-   * <code>optional string json_name = 10;</code>
+   * <code>string json_name = 10;</code>
    */
   java.lang.String getJsonName();
   /**
@@ -164,7 +164,7 @@ public interface FieldOrBuilder extends
    * The field JSON name.
    * </pre>
    *
-   * <code>optional string json_name = 10;</code>
+   * <code>string json_name = 10;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getJsonNameBytes();
@@ -174,7 +174,7 @@ public interface FieldOrBuilder extends
    * The string value of the default value of this field. Proto2 syntax only.
    * </pre>
    *
-   * <code>optional string default_value = 11;</code>
+   * <code>string default_value = 11;</code>
    */
   java.lang.String getDefaultValue();
   /**
@@ -182,7 +182,7 @@ public interface FieldOrBuilder extends
    * The string value of the default value of this field. Proto2 syntax only.
    * </pre>
    *
-   * <code>optional string default_value = 11;</code>
+   * <code>string default_value = 11;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getDefaultValueBytes();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldSet.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldSet.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldSet.java
index 36a74ed..a1bf553 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldSet.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FieldSet.java
@@ -220,6 +220,7 @@ final class FieldSet<FieldDescriptorType extends
     return fields.entrySet().iterator();
   }
 
+
   /**
    * Useful for implementing
    * {@link Message#hasField(Descriptors.FieldDescriptor)}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValue.java
index 656ebb6..f3315c8 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValue.java
@@ -83,7 +83,7 @@ public  final class FloatValue extends
    * The float value.
    * </pre>
    *
-   * <code>optional float value = 1;</code>
+   * <code>float value = 1;</code>
    */
   public float getValue() {
     return value_;
@@ -144,7 +144,7 @@ public  final class FloatValue extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + java.lang.Float.floatToIntBits(
         getValue());
@@ -372,7 +372,7 @@ public  final class FloatValue extends
      * The float value.
      * </pre>
      *
-     * <code>optional float value = 1;</code>
+     * <code>float value = 1;</code>
      */
     public float getValue() {
       return value_;
@@ -382,7 +382,7 @@ public  final class FloatValue extends
      * The float value.
      * </pre>
      *
-     * <code>optional float value = 1;</code>
+     * <code>float value = 1;</code>
      */
     public Builder setValue(float value) {
       
@@ -395,7 +395,7 @@ public  final class FloatValue extends
      * The float value.
      * </pre>
      *
-     * <code>optional float value = 1;</code>
+     * <code>float value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValueOrBuilder.java
index db330e8..a5302d7 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/FloatValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface FloatValueOrBuilder extends
    * The float value.
    * </pre>
    *
-   * <code>optional float value = 1;</code>
+   * <code>float value = 1;</code>
    */
   float getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageLite.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageLite.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageLite.java
index 057100b..584eba1 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageLite.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageLite.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.IntList;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.LongList;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.ProtobufList;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.WireFormat.FieldType;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.ObjectStreamException;
@@ -479,7 +478,6 @@ public abstract class GeneratedMessageLite<
         CodedInputStream input,
         ExtensionRegistryLite extensionRegistry,
         int tag) throws IOException {
-      int wireType = WireFormat.getTagWireType(tag);
       int fieldNumber = WireFormat.getTagFieldNumber(tag);
 
       // TODO(dweis): How much bytecode would be saved by not requiring the generated code to
@@ -487,6 +485,17 @@ public abstract class GeneratedMessageLite<
       GeneratedExtension<MessageType, ?> extension = extensionRegistry.findLiteExtensionByNumber(
           defaultInstance, fieldNumber);
 
+      return parseExtension(input, extensionRegistry, extension, tag, fieldNumber);
+    }
+
+    private boolean parseExtension(
+        CodedInputStream input,
+        ExtensionRegistryLite extensionRegistry,
+        GeneratedExtension<?, ?> extension,
+        int tag,
+        int fieldNumber)
+        throws IOException {
+      int wireType = WireFormat.getTagWireType(tag);
       boolean unknown = false;
       boolean packed = false;
       if (extension == null) {
@@ -508,7 +517,7 @@ public abstract class GeneratedMessageLite<
       if (unknown) {  // Unknown field or wrong wire type.  Skip.
         return parseUnknownField(tag, input);
       }
-
+      
       if (packed) {
         int length = input.readRawVarint32();
         int limit = input.pushLimit(length);
@@ -587,9 +596,147 @@ public abstract class GeneratedMessageLite<
                               extension.singularToFieldSetType(value));
         }
       }
-
       return true;
     }
+    
+    /**
+     * Parse an unknown field or an extension. For use by generated code only.
+     *
+     * <p>For use by generated code only.
+     *
+     * @return {@code true} unless the tag is an end-group tag.
+     */
+    protected <MessageType extends MessageLite> boolean parseUnknownFieldAsMessageSet(
+        MessageType defaultInstance,
+        CodedInputStream input,
+        ExtensionRegistryLite extensionRegistry,
+        int tag)
+        throws IOException {
+
+      if (tag == WireFormat.MESSAGE_SET_ITEM_TAG) {
+        mergeMessageSetExtensionFromCodedStream(defaultInstance, input, extensionRegistry);
+        return true;
+      }
+
+      // TODO(dweis): Do we really want to support non message set wire format in message sets?
+      // Full runtime does... So we do for now.
+      int wireType = WireFormat.getTagWireType(tag);
+      if (wireType == WireFormat.WIRETYPE_LENGTH_DELIMITED) {
+        return parseUnknownField(defaultInstance, input, extensionRegistry, tag);
+      } else {
+        // TODO(dweis): Should we throw on invalid input? Full runtime does not...
+        return input.skipField(tag);
+      }
+    }
+
+    /**
+     * Merges the message set from the input stream; requires message set wire format.
+     * 
+     * @param defaultInstance the default instance of the containing message we are parsing in
+     * @param input the stream to parse from
+     * @param extensionRegistry the registry to use when parsing
+     */
+    private <MessageType extends MessageLite> void mergeMessageSetExtensionFromCodedStream(
+        MessageType defaultInstance,
+        CodedInputStream input,
+        ExtensionRegistryLite extensionRegistry)
+        throws IOException {
+      // The wire format for MessageSet is:
+      //   message MessageSet {
+      //     repeated group Item = 1 {
+      //       required int32 typeId = 2;
+      //       required bytes message = 3;
+      //     }
+      //   }
+      // "typeId" is the extension's field number.  The extension can only be
+      // a message type, where "message" contains the encoded bytes of that
+      // message.
+      //
+      // In practice, we will probably never see a MessageSet item in which
+      // the message appears before the type ID, or where either field does not
+      // appear exactly once.  However, in theory such cases are valid, so we
+      // should be prepared to accept them.
+
+      int typeId = 0;
+      ByteString rawBytes = null; // If we encounter "message" before "typeId"
+      GeneratedExtension<?, ?> extension = null;
+
+      // Read bytes from input, if we get it's type first then parse it eagerly,
+      // otherwise we store the raw bytes in a local variable.
+      while (true) {
+        final int tag = input.readTag();
+        if (tag == 0) {
+          break;
+        }
+
+        if (tag == WireFormat.MESSAGE_SET_TYPE_ID_TAG) {
+          typeId = input.readUInt32();
+          if (typeId != 0) {
+            extension = extensionRegistry.findLiteExtensionByNumber(defaultInstance, typeId);
+          }
+
+        } else if (tag == WireFormat.MESSAGE_SET_MESSAGE_TAG) {
+          if (typeId != 0) {
+            if (extension != null) {
+              // We already know the type, so we can parse directly from the
+              // input with no copying.  Hooray!
+              eagerlyMergeMessageSetExtension(input, extension, extensionRegistry, typeId);
+              rawBytes = null;
+              continue;
+            }
+          }
+          // We haven't seen a type ID yet or we want parse message lazily.
+          rawBytes = input.readBytes();
+
+        } else { // Unknown tag. Skip it.
+          if (!input.skipField(tag)) {
+            break; // End of group
+          }
+        }
+      }
+      input.checkLastTagWas(WireFormat.MESSAGE_SET_ITEM_END_TAG);
+
+      // Process the raw bytes.
+      if (rawBytes != null && typeId != 0) { // Zero is not a valid type ID.
+        if (extension != null) { // We known the type
+          mergeMessageSetExtensionFromBytes(rawBytes, extensionRegistry, extension);
+        } else { // We don't know how to parse this. Ignore it.
+          if (rawBytes != null) {
+            mergeLengthDelimitedField(typeId, rawBytes);
+          }
+        }
+      }
+    }
+
+    private void eagerlyMergeMessageSetExtension(
+        CodedInputStream input,
+        GeneratedExtension<?, ?> extension,
+        ExtensionRegistryLite extensionRegistry,
+        int typeId)
+        throws IOException {
+      int fieldNumber = typeId;
+      int tag = WireFormat.makeTag(typeId, WireFormat.WIRETYPE_LENGTH_DELIMITED);
+      parseExtension(input, extensionRegistry, extension, tag, fieldNumber);
+    }
+
+    private void mergeMessageSetExtensionFromBytes(
+        ByteString rawBytes,
+        ExtensionRegistryLite extensionRegistry,
+        GeneratedExtension<?, ?> extension)
+        throws IOException {
+      MessageLite.Builder subBuilder = null;
+      MessageLite existingValue = (MessageLite) extensions.getField(extension.descriptor);
+      if (existingValue != null) {
+        subBuilder = existingValue.toBuilder();
+      }
+      if (subBuilder == null) {
+        subBuilder = extension.getMessageDefaultInstance().newBuilderForType();
+      }
+      rawBytes.newCodedInput().readMessage(subBuilder, extensionRegistry);
+      MessageLite value = subBuilder.build();
+
+      extensions.setField(extension.descriptor, extension.singularToFieldSetType(value));
+    }
 
     private void verifyExtensionContainingType(
         final GeneratedExtension<MessageType, ?> extension) {
@@ -807,14 +954,6 @@ public abstract class GeneratedMessageLite<
       return instance.getExtension(extension, index);
     }
 
-    // This is implemented here only to work around an apparent bug in the
-    // Java compiler and/or build system.  See bug #1898463.  The mere presence
-    // of this dummy clone() implementation makes it go away.
-    @Override
-    public BuilderType clone() {
-      return super.clone();
-    }
-
     /** Set the value of an extension. */
     public final <Type> BuilderType setExtension(
         final ExtensionLite<MessageType, Type> extension,

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageV3.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageV3.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageV3.java
index fe92e6f..cc815fc 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageV3.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/GeneratedMessageV3.java
@@ -36,6 +36,16 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueD
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor;
+// In opensource protobuf, we have versioned this GeneratedMessageV3 class to GeneratedMessageV3V3 and
+// in the future may have GeneratedMessageV3V4 etc. This allows us to change some aspects of this
+// class without breaking binary compatibility with old generated code that still subclasses
+// the old GeneratedMessageV3 class. To allow these different GeneratedMessageV3V? classes to
+// interoperate (e.g., a GeneratedMessageV3V3 object has a message extension field whose class
+// type is GeneratedMessageV3V4), these classes still share a common parent class AbstarctMessage
+// and are using the same GeneratedMessage.GeneratedExtension class for extension definitions.
+// Since this class becomes GeneratedMessageV3V? in opensource, we have to add an import here
+// to be able to use GeneratedMessage.GeneratedExtension. The GeneratedExtension definition in
+// this file is also excluded from opensource to avoid conflict.
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessage.GeneratedExtension;
 
 import java.io.IOException;
@@ -1207,14 +1217,6 @@ public abstract class GeneratedMessageV3 extends AbstractMessage
       return super.clear();
     }
 
-    // This is implemented here only to work around an apparent bug in the
-    // Java compiler and/or build system.  See bug #1898463.  The mere presence
-    // of this clone() implementation makes it go away.
-    @Override
-    public BuilderType clone() {
-      return super.clone();
-    }
-
     private void ensureExtensionsIsMutable() {
       if (extensions.isImmutable()) {
         extensions = extensions.clone();
@@ -1610,6 +1612,7 @@ public abstract class GeneratedMessageV3 extends AbstractMessage
     FieldDescriptor getDescriptor();
   }
 
+
   // =================================================================
 
   /** Calls Class.getMethod and throws a RuntimeException if it fails. */
@@ -1705,11 +1708,6 @@ public abstract class GeneratedMessageV3 extends AbstractMessage
       initialized = false;
     }
 
-    private boolean isMapFieldEnabled(FieldDescriptor field) {
-      boolean result = true;
-      return result;
-    }
-
     /**
      * Ensures the field accessors are initialized. This method is thread-safe.
      *
@@ -1733,7 +1731,7 @@ public abstract class GeneratedMessageV3 extends AbstractMessage
           }
           if (field.isRepeated()) {
             if (field.getJavaType() == FieldDescriptor.JavaType.MESSAGE) {
-              if (field.isMapField() && isMapFieldEnabled(field)) {
+              if (field.isMapField()) {
                 fields[i] = new MapFieldAccessor(
                     field, camelCaseNames[i], messageClass, builderClass);
               } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32Value.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32Value.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32Value.java
index 89db936..476b086 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32Value.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32Value.java
@@ -83,7 +83,7 @@ public  final class Int32Value extends
    * The int32 value.
    * </pre>
    *
-   * <code>optional int32 value = 1;</code>
+   * <code>int32 value = 1;</code>
    */
   public int getValue() {
     return value_;
@@ -142,7 +142,7 @@ public  final class Int32Value extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + getValue();
     hash = (29 * hash) + unknownFields.hashCode();
@@ -369,7 +369,7 @@ public  final class Int32Value extends
      * The int32 value.
      * </pre>
      *
-     * <code>optional int32 value = 1;</code>
+     * <code>int32 value = 1;</code>
      */
     public int getValue() {
       return value_;
@@ -379,7 +379,7 @@ public  final class Int32Value extends
      * The int32 value.
      * </pre>
      *
-     * <code>optional int32 value = 1;</code>
+     * <code>int32 value = 1;</code>
      */
     public Builder setValue(int value) {
       
@@ -392,7 +392,7 @@ public  final class Int32Value extends
      * The int32 value.
      * </pre>
      *
-     * <code>optional int32 value = 1;</code>
+     * <code>int32 value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32ValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32ValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32ValueOrBuilder.java
index df1eb5f..f7b0b25 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32ValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int32ValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface Int32ValueOrBuilder extends
    * The int32 value.
    * </pre>
    *
-   * <code>optional int32 value = 1;</code>
+   * <code>int32 value = 1;</code>
    */
   int getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64Value.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64Value.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64Value.java
index 7ab34ff..fce2b73 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64Value.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64Value.java
@@ -83,7 +83,7 @@ public  final class Int64Value extends
    * The int64 value.
    * </pre>
    *
-   * <code>optional int64 value = 1;</code>
+   * <code>int64 value = 1;</code>
    */
   public long getValue() {
     return value_;
@@ -142,7 +142,7 @@ public  final class Int64Value extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
         getValue());
@@ -370,7 +370,7 @@ public  final class Int64Value extends
      * The int64 value.
      * </pre>
      *
-     * <code>optional int64 value = 1;</code>
+     * <code>int64 value = 1;</code>
      */
     public long getValue() {
       return value_;
@@ -380,7 +380,7 @@ public  final class Int64Value extends
      * The int64 value.
      * </pre>
      *
-     * <code>optional int64 value = 1;</code>
+     * <code>int64 value = 1;</code>
      */
     public Builder setValue(long value) {
       
@@ -393,7 +393,7 @@ public  final class Int64Value extends
      * The int64 value.
      * </pre>
      *
-     * <code>optional int64 value = 1;</code>
+     * <code>int64 value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64ValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64ValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64ValueOrBuilder.java
index 8cbe470..291f5ed 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64ValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Int64ValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface Int64ValueOrBuilder extends
    * The int64 value.
    * </pre>
    *
-   * <code>optional int64 value = 1;</code>
+   * <code>int64 value = 1;</code>
    */
   long getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/LazyFieldLite.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/LazyFieldLite.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/LazyFieldLite.java
index 4d4ac16..91b70e0 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/LazyFieldLite.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/LazyFieldLite.java
@@ -284,29 +284,8 @@ public class LazyFieldLite {
       return;
     }
 
-    // At this point we have two fully parsed messages. We can't merge directly from one to the
-    // other because only generated builder code contains methods to mergeFrom another parsed
-    // message. We have to serialize one instance and then merge the bytes into the other. This may
-    // drop extensions from one of the messages if one of the values had an extension set on it
-    // directly.
-    //
-    // To mitigate this we prefer serializing a message that has an extension registry, and
-    // therefore a chance that all extensions set on it are in that registry.
-    //
-    // NOTE: The check for other.extensionRegistry not being null must come first because at this
-    // point in time if other.extensionRegistry is not null then this.extensionRegistry will not be
-    // null either.
-    if (other.extensionRegistry != null) {
-      setValue(mergeValueAndBytes(this.value, other.toByteString(), other.extensionRegistry));
-      return;
-    } else if (this.extensionRegistry != null) {
-      setValue(mergeValueAndBytes(other.value, this.toByteString(), this.extensionRegistry));
-      return;
-    } else {
-      // All extensions from the other message will be dropped because we have no registry.
-      setValue(mergeValueAndBytes(this.value, other.toByteString(), EMPTY_REGISTRY));
-      return;
-    }
+    // At this point we have two fully parsed messages.
+    setValue(this.value.toBuilder().mergeFrom(other.value).build());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ListValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ListValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ListValue.java
index 560130d..4ddc71b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ListValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ListValue.java
@@ -191,7 +191,7 @@ public  final class ListValue extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     if (getValuesCount() > 0) {
       hash = (37 * hash) + VALUES_FIELD_NUMBER;
       hash = (53 * hash) + getValuesList().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapEntry.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapEntry.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapEntry.java
index addc980..d586cc7 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapEntry.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapEntry.java
@@ -109,7 +109,7 @@ public final class MapEntry<K, V> extends AbstractMessage {
     } catch (InvalidProtocolBufferException e) {
       throw e.setUnfinishedMessage(this);
     } catch (IOException e) {
-      throw new InvalidProtocolBufferException(e.getMessage()).setUnfinishedMessage(this);
+      throw new InvalidProtocolBufferException(e).setUnfinishedMessage(this);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapFieldLite.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapFieldLite.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapFieldLite.java
index 414a34e..761c4ce 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapFieldLite.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MapFieldLite.java
@@ -58,7 +58,7 @@ public final class MapFieldLite<K, V> extends LinkedHashMap<K, V> {
   }
 
   @SuppressWarnings({"rawtypes", "unchecked"})
-  private static final MapFieldLite EMPTY_MAP_FIELD = new MapFieldLite(Collections.emptyMap());
+  private static final MapFieldLite EMPTY_MAP_FIELD = new MapFieldLite();
   static {
     EMPTY_MAP_FIELD.makeImmutable();
   }
@@ -83,7 +83,7 @@ public final class MapFieldLite<K, V> extends LinkedHashMap<K, V> {
 
   @Override public void clear() {
     ensureMutable();
-    clear();
+    super.clear();
   }
 
   @Override public V put(K key, V value) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MessageLiteToString.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MessageLiteToString.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MessageLiteToString.java
index 51ff83a..eea6db5 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MessageLiteToString.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MessageLiteToString.java
@@ -95,7 +95,7 @@ final class MessageLiteToString {
         // Try to reflectively get the value and toString() the field as if it were repeated. This
         // only works if the method names have not be proguarded out or renamed.
         Method listMethod = nameToNoArgMethod.get("get" + suffix);
-        if (listMethod != null) {
+        if (listMethod != null && listMethod.getReturnType().equals(List.class)) {
           printField(
               buffer,
               indent,
@@ -115,7 +115,7 @@ final class MessageLiteToString {
         // Heuristic to skip bytes based accessors for string fields.
         continue;
       }
-      
+
       String camelCase = suffix.substring(0, 1).toLowerCase() + suffix.substring(1);
 
       // Try to reflectively get the value and toString() the field as if it were optional. This

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Method.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Method.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Method.java
index 5af69eb..32e8b35 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Method.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Method.java
@@ -130,7 +130,7 @@ public  final class Method extends
    * The simple name of this method.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -149,7 +149,7 @@ public  final class Method extends
    * The simple name of this method.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -172,7 +172,7 @@ public  final class Method extends
    * A URL of the input message type.
    * </pre>
    *
-   * <code>optional string request_type_url = 2;</code>
+   * <code>string request_type_url = 2;</code>
    */
   public java.lang.String getRequestTypeUrl() {
     java.lang.Object ref = requestTypeUrl_;
@@ -191,7 +191,7 @@ public  final class Method extends
    * A URL of the input message type.
    * </pre>
    *
-   * <code>optional string request_type_url = 2;</code>
+   * <code>string request_type_url = 2;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getRequestTypeUrlBytes() {
@@ -214,7 +214,7 @@ public  final class Method extends
    * If true, the request is streamed.
    * </pre>
    *
-   * <code>optional bool request_streaming = 3;</code>
+   * <code>bool request_streaming = 3;</code>
    */
   public boolean getRequestStreaming() {
     return requestStreaming_;
@@ -227,7 +227,7 @@ public  final class Method extends
    * The URL of the output message type.
    * </pre>
    *
-   * <code>optional string response_type_url = 4;</code>
+   * <code>string response_type_url = 4;</code>
    */
   public java.lang.String getResponseTypeUrl() {
     java.lang.Object ref = responseTypeUrl_;
@@ -246,7 +246,7 @@ public  final class Method extends
    * The URL of the output message type.
    * </pre>
    *
-   * <code>optional string response_type_url = 4;</code>
+   * <code>string response_type_url = 4;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getResponseTypeUrlBytes() {
@@ -269,7 +269,7 @@ public  final class Method extends
    * If true, the response is streamed.
    * </pre>
    *
-   * <code>optional bool response_streaming = 5;</code>
+   * <code>bool response_streaming = 5;</code>
    */
   public boolean getResponseStreaming() {
     return responseStreaming_;
@@ -337,7 +337,7 @@ public  final class Method extends
    * The source syntax of this method.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   public int getSyntaxValue() {
     return syntax_;
@@ -347,7 +347,7 @@ public  final class Method extends
    * The source syntax of this method.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -457,7 +457,7 @@ public  final class Method extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + NAME_FIELD_NUMBER;
     hash = (53 * hash) + getName().hashCode();
     hash = (37 * hash) + REQUEST_TYPE_URL_FIELD_NUMBER;
@@ -778,7 +778,7 @@ public  final class Method extends
      * The simple name of this method.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -797,7 +797,7 @@ public  final class Method extends
      * The simple name of this method.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -817,7 +817,7 @@ public  final class Method extends
      * The simple name of this method.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -834,7 +834,7 @@ public  final class Method extends
      * The simple name of this method.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder clearName() {
       
@@ -847,7 +847,7 @@ public  final class Method extends
      * The simple name of this method.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -867,7 +867,7 @@ public  final class Method extends
      * A URL of the input message type.
      * </pre>
      *
-     * <code>optional string request_type_url = 2;</code>
+     * <code>string request_type_url = 2;</code>
      */
     public java.lang.String getRequestTypeUrl() {
       java.lang.Object ref = requestTypeUrl_;
@@ -886,7 +886,7 @@ public  final class Method extends
      * A URL of the input message type.
      * </pre>
      *
-     * <code>optional string request_type_url = 2;</code>
+     * <code>string request_type_url = 2;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRequestTypeUrlBytes() {
@@ -906,7 +906,7 @@ public  final class Method extends
      * A URL of the input message type.
      * </pre>
      *
-     * <code>optional string request_type_url = 2;</code>
+     * <code>string request_type_url = 2;</code>
      */
     public Builder setRequestTypeUrl(
         java.lang.String value) {
@@ -923,7 +923,7 @@ public  final class Method extends
      * A URL of the input message type.
      * </pre>
      *
-     * <code>optional string request_type_url = 2;</code>
+     * <code>string request_type_url = 2;</code>
      */
     public Builder clearRequestTypeUrl() {
       
@@ -936,7 +936,7 @@ public  final class Method extends
      * A URL of the input message type.
      * </pre>
      *
-     * <code>optional string request_type_url = 2;</code>
+     * <code>string request_type_url = 2;</code>
      */
     public Builder setRequestTypeUrlBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -956,7 +956,7 @@ public  final class Method extends
      * If true, the request is streamed.
      * </pre>
      *
-     * <code>optional bool request_streaming = 3;</code>
+     * <code>bool request_streaming = 3;</code>
      */
     public boolean getRequestStreaming() {
       return requestStreaming_;
@@ -966,7 +966,7 @@ public  final class Method extends
      * If true, the request is streamed.
      * </pre>
      *
-     * <code>optional bool request_streaming = 3;</code>
+     * <code>bool request_streaming = 3;</code>
      */
     public Builder setRequestStreaming(boolean value) {
       
@@ -979,7 +979,7 @@ public  final class Method extends
      * If true, the request is streamed.
      * </pre>
      *
-     * <code>optional bool request_streaming = 3;</code>
+     * <code>bool request_streaming = 3;</code>
      */
     public Builder clearRequestStreaming() {
       
@@ -994,7 +994,7 @@ public  final class Method extends
      * The URL of the output message type.
      * </pre>
      *
-     * <code>optional string response_type_url = 4;</code>
+     * <code>string response_type_url = 4;</code>
      */
     public java.lang.String getResponseTypeUrl() {
       java.lang.Object ref = responseTypeUrl_;
@@ -1013,7 +1013,7 @@ public  final class Method extends
      * The URL of the output message type.
      * </pre>
      *
-     * <code>optional string response_type_url = 4;</code>
+     * <code>string response_type_url = 4;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getResponseTypeUrlBytes() {
@@ -1033,7 +1033,7 @@ public  final class Method extends
      * The URL of the output message type.
      * </pre>
      *
-     * <code>optional string response_type_url = 4;</code>
+     * <code>string response_type_url = 4;</code>
      */
     public Builder setResponseTypeUrl(
         java.lang.String value) {
@@ -1050,7 +1050,7 @@ public  final class Method extends
      * The URL of the output message type.
      * </pre>
      *
-     * <code>optional string response_type_url = 4;</code>
+     * <code>string response_type_url = 4;</code>
      */
     public Builder clearResponseTypeUrl() {
       
@@ -1063,7 +1063,7 @@ public  final class Method extends
      * The URL of the output message type.
      * </pre>
      *
-     * <code>optional string response_type_url = 4;</code>
+     * <code>string response_type_url = 4;</code>
      */
     public Builder setResponseTypeUrlBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -1083,7 +1083,7 @@ public  final class Method extends
      * If true, the response is streamed.
      * </pre>
      *
-     * <code>optional bool response_streaming = 5;</code>
+     * <code>bool response_streaming = 5;</code>
      */
     public boolean getResponseStreaming() {
       return responseStreaming_;
@@ -1093,7 +1093,7 @@ public  final class Method extends
      * If true, the response is streamed.
      * </pre>
      *
-     * <code>optional bool response_streaming = 5;</code>
+     * <code>bool response_streaming = 5;</code>
      */
     public Builder setResponseStreaming(boolean value) {
       
@@ -1106,7 +1106,7 @@ public  final class Method extends
      * If true, the response is streamed.
      * </pre>
      *
-     * <code>optional bool response_streaming = 5;</code>
+     * <code>bool response_streaming = 5;</code>
      */
     public Builder clearResponseStreaming() {
       
@@ -1433,7 +1433,7 @@ public  final class Method extends
      * The source syntax of this method.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public int getSyntaxValue() {
       return syntax_;
@@ -1443,7 +1443,7 @@ public  final class Method extends
      * The source syntax of this method.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public Builder setSyntaxValue(int value) {
       syntax_ = value;
@@ -1455,7 +1455,7 @@ public  final class Method extends
      * The source syntax of this method.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -1466,7 +1466,7 @@ public  final class Method extends
      * The source syntax of this method.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public Builder setSyntax(org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax value) {
       if (value == null) {
@@ -1482,7 +1482,7 @@ public  final class Method extends
      * The source syntax of this method.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public Builder clearSyntax() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MethodOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MethodOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MethodOrBuilder.java
index b40776c..12819df 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MethodOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MethodOrBuilder.java
@@ -12,7 +12,7 @@ public interface MethodOrBuilder extends
    * The simple name of this method.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   java.lang.String getName();
   /**
@@ -20,7 +20,7 @@ public interface MethodOrBuilder extends
    * The simple name of this method.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
@@ -30,7 +30,7 @@ public interface MethodOrBuilder extends
    * A URL of the input message type.
    * </pre>
    *
-   * <code>optional string request_type_url = 2;</code>
+   * <code>string request_type_url = 2;</code>
    */
   java.lang.String getRequestTypeUrl();
   /**
@@ -38,7 +38,7 @@ public interface MethodOrBuilder extends
    * A URL of the input message type.
    * </pre>
    *
-   * <code>optional string request_type_url = 2;</code>
+   * <code>string request_type_url = 2;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getRequestTypeUrlBytes();
@@ -48,7 +48,7 @@ public interface MethodOrBuilder extends
    * If true, the request is streamed.
    * </pre>
    *
-   * <code>optional bool request_streaming = 3;</code>
+   * <code>bool request_streaming = 3;</code>
    */
   boolean getRequestStreaming();
 
@@ -57,7 +57,7 @@ public interface MethodOrBuilder extends
    * The URL of the output message type.
    * </pre>
    *
-   * <code>optional string response_type_url = 4;</code>
+   * <code>string response_type_url = 4;</code>
    */
   java.lang.String getResponseTypeUrl();
   /**
@@ -65,7 +65,7 @@ public interface MethodOrBuilder extends
    * The URL of the output message type.
    * </pre>
    *
-   * <code>optional string response_type_url = 4;</code>
+   * <code>string response_type_url = 4;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getResponseTypeUrlBytes();
@@ -75,7 +75,7 @@ public interface MethodOrBuilder extends
    * If true, the response is streamed.
    * </pre>
    *
-   * <code>optional bool response_streaming = 5;</code>
+   * <code>bool response_streaming = 5;</code>
    */
   boolean getResponseStreaming();
 
@@ -128,7 +128,7 @@ public interface MethodOrBuilder extends
    * The source syntax of this method.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   int getSyntaxValue();
   /**
@@ -136,7 +136,7 @@ public interface MethodOrBuilder extends
    * The source syntax of this method.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Mixin.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Mixin.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Mixin.java
index 0d9fe46..5dd6be6 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Mixin.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Mixin.java
@@ -150,7 +150,7 @@ public  final class Mixin extends
    * The fully qualified name of the API which is included.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -169,7 +169,7 @@ public  final class Mixin extends
    * The fully qualified name of the API which is included.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -193,7 +193,7 @@ public  final class Mixin extends
    * are rooted.
    * </pre>
    *
-   * <code>optional string root = 2;</code>
+   * <code>string root = 2;</code>
    */
   public java.lang.String getRoot() {
     java.lang.Object ref = root_;
@@ -213,7 +213,7 @@ public  final class Mixin extends
    * are rooted.
    * </pre>
    *
-   * <code>optional string root = 2;</code>
+   * <code>string root = 2;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getRootBytes() {
@@ -289,7 +289,7 @@ public  final class Mixin extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + NAME_FIELD_NUMBER;
     hash = (53 * hash) + getName().hashCode();
     hash = (37 * hash) + ROOT_FIELD_NUMBER;
@@ -585,7 +585,7 @@ public  final class Mixin extends
      * The fully qualified name of the API which is included.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -604,7 +604,7 @@ public  final class Mixin extends
      * The fully qualified name of the API which is included.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -624,7 +624,7 @@ public  final class Mixin extends
      * The fully qualified name of the API which is included.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -641,7 +641,7 @@ public  final class Mixin extends
      * The fully qualified name of the API which is included.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder clearName() {
       
@@ -654,7 +654,7 @@ public  final class Mixin extends
      * The fully qualified name of the API which is included.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -675,7 +675,7 @@ public  final class Mixin extends
      * are rooted.
      * </pre>
      *
-     * <code>optional string root = 2;</code>
+     * <code>string root = 2;</code>
      */
     public java.lang.String getRoot() {
       java.lang.Object ref = root_;
@@ -695,7 +695,7 @@ public  final class Mixin extends
      * are rooted.
      * </pre>
      *
-     * <code>optional string root = 2;</code>
+     * <code>string root = 2;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getRootBytes() {
@@ -716,7 +716,7 @@ public  final class Mixin extends
      * are rooted.
      * </pre>
      *
-     * <code>optional string root = 2;</code>
+     * <code>string root = 2;</code>
      */
     public Builder setRoot(
         java.lang.String value) {
@@ -734,7 +734,7 @@ public  final class Mixin extends
      * are rooted.
      * </pre>
      *
-     * <code>optional string root = 2;</code>
+     * <code>string root = 2;</code>
      */
     public Builder clearRoot() {
       
@@ -748,7 +748,7 @@ public  final class Mixin extends
      * are rooted.
      * </pre>
      *
-     * <code>optional string root = 2;</code>
+     * <code>string root = 2;</code>
      */
     public Builder setRootBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MixinOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MixinOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MixinOrBuilder.java
index 8d9311f..99e31c6 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MixinOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/MixinOrBuilder.java
@@ -12,7 +12,7 @@ public interface MixinOrBuilder extends
    * The fully qualified name of the API which is included.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   java.lang.String getName();
   /**
@@ -20,7 +20,7 @@ public interface MixinOrBuilder extends
    * The fully qualified name of the API which is included.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
@@ -31,7 +31,7 @@ public interface MixinOrBuilder extends
    * are rooted.
    * </pre>
    *
-   * <code>optional string root = 2;</code>
+   * <code>string root = 2;</code>
    */
   java.lang.String getRoot();
   /**
@@ -40,7 +40,7 @@ public interface MixinOrBuilder extends
    * are rooted.
    * </pre>
    *
-   * <code>optional string root = 2;</code>
+   * <code>string root = 2;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getRootBytes();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/NullValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/NullValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/NullValue.java
index 38762a4..6cedb68 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/NullValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/NullValue.java
@@ -80,8 +80,7 @@ public enum NullValue
   }
   public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
       getDescriptor() {
-    return org.apache.hadoop.hbase.shaded.com.google.protobuf.StructProto.getDescriptor()
-        .getEnumTypes().get(0);
+    return org.apache.hadoop.hbase.shaded.com.google.protobuf.StructProto.getDescriptor().getEnumTypes().get(0);
   }
 
   private static final NullValue[] VALUES = values();


[04/50] [abbrv] hbase git commit: HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/compiler/PluginProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/compiler/PluginProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/compiler/PluginProtos.java
index 42627bd..71975c2 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/compiler/PluginProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/compiler/PluginProtos.java
@@ -14,6 +14,879 @@ public final class PluginProtos {
     registerAllExtensions(
         (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
+  public interface VersionOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:google.protobuf.compiler.Version)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional int32 major = 1;</code>
+     */
+    boolean hasMajor();
+    /**
+     * <code>optional int32 major = 1;</code>
+     */
+    int getMajor();
+
+    /**
+     * <code>optional int32 minor = 2;</code>
+     */
+    boolean hasMinor();
+    /**
+     * <code>optional int32 minor = 2;</code>
+     */
+    int getMinor();
+
+    /**
+     * <code>optional int32 patch = 3;</code>
+     */
+    boolean hasPatch();
+    /**
+     * <code>optional int32 patch = 3;</code>
+     */
+    int getPatch();
+
+    /**
+     * <pre>
+     * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+     * be empty for mainline stable releases.
+     * </pre>
+     *
+     * <code>optional string suffix = 4;</code>
+     */
+    boolean hasSuffix();
+    /**
+     * <pre>
+     * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+     * be empty for mainline stable releases.
+     * </pre>
+     *
+     * <code>optional string suffix = 4;</code>
+     */
+    java.lang.String getSuffix();
+    /**
+     * <pre>
+     * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+     * be empty for mainline stable releases.
+     * </pre>
+     *
+     * <code>optional string suffix = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getSuffixBytes();
+  }
+  /**
+   * <pre>
+   * The version number of protocol compiler.
+   * </pre>
+   *
+   * Protobuf type {@code google.protobuf.compiler.Version}
+   */
+  public  static final class Version extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:google.protobuf.compiler.Version)
+      VersionOrBuilder {
+    // Use Version.newBuilder() to construct.
+    private Version(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private Version() {
+      major_ = 0;
+      minor_ = 0;
+      patch_ = 0;
+      suffix_ = "";
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private Version(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.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;
+              major_ = input.readInt32();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              minor_ = input.readInt32();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              patch_ = input.readInt32();
+              break;
+            }
+            case 34: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000008;
+              suffix_ = bs;
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.internal_static_google_protobuf_compiler_Version_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.internal_static_google_protobuf_compiler_Version_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.class, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int MAJOR_FIELD_NUMBER = 1;
+    private int major_;
+    /**
+     * <code>optional int32 major = 1;</code>
+     */
+    public boolean hasMajor() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 major = 1;</code>
+     */
+    public int getMajor() {
+      return major_;
+    }
+
+    public static final int MINOR_FIELD_NUMBER = 2;
+    private int minor_;
+    /**
+     * <code>optional int32 minor = 2;</code>
+     */
+    public boolean hasMinor() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional int32 minor = 2;</code>
+     */
+    public int getMinor() {
+      return minor_;
+    }
+
+    public static final int PATCH_FIELD_NUMBER = 3;
+    private int patch_;
+    /**
+     * <code>optional int32 patch = 3;</code>
+     */
+    public boolean hasPatch() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int32 patch = 3;</code>
+     */
+    public int getPatch() {
+      return patch_;
+    }
+
+    public static final int SUFFIX_FIELD_NUMBER = 4;
+    private volatile java.lang.Object suffix_;
+    /**
+     * <pre>
+     * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+     * be empty for mainline stable releases.
+     * </pre>
+     *
+     * <code>optional string suffix = 4;</code>
+     */
+    public boolean hasSuffix() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <pre>
+     * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+     * be empty for mainline stable releases.
+     * </pre>
+     *
+     * <code>optional string suffix = 4;</code>
+     */
+    public java.lang.String getSuffix() {
+      java.lang.Object ref = suffix_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          suffix_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <pre>
+     * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+     * be empty for mainline stable releases.
+     * </pre>
+     *
+     * <code>optional string suffix = 4;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getSuffixBytes() {
+      java.lang.Object ref = suffix_;
+      if (ref instanceof java.lang.String) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        suffix_ = b;
+        return b;
+      } else {
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(1, major_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeInt32(2, minor_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt32(3, patch_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 4, suffix_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeInt32Size(1, major_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, minor_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeInt32Size(3, patch_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(4, suffix_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version other = (org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version) obj;
+
+      boolean result = true;
+      result = result && (hasMajor() == other.hasMajor());
+      if (hasMajor()) {
+        result = result && (getMajor()
+            == other.getMajor());
+      }
+      result = result && (hasMinor() == other.hasMinor());
+      if (hasMinor()) {
+        result = result && (getMinor()
+            == other.getMinor());
+      }
+      result = result && (hasPatch() == other.hasPatch());
+      if (hasPatch()) {
+        result = result && (getPatch()
+            == other.getPatch());
+      }
+      result = result && (hasSuffix() == other.hasSuffix());
+      if (hasSuffix()) {
+        result = result && getSuffix()
+            .equals(other.getSuffix());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasMajor()) {
+        hash = (37 * hash) + MAJOR_FIELD_NUMBER;
+        hash = (53 * hash) + getMajor();
+      }
+      if (hasMinor()) {
+        hash = (37 * hash) + MINOR_FIELD_NUMBER;
+        hash = (53 * hash) + getMinor();
+      }
+      if (hasPatch()) {
+        hash = (37 * hash) + PATCH_FIELD_NUMBER;
+        hash = (53 * hash) + getPatch();
+      }
+      if (hasSuffix()) {
+        hash = (37 * hash) + SUFFIX_FIELD_NUMBER;
+        hash = (53 * hash) + getSuffix().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     * The version number of protocol compiler.
+     * </pre>
+     *
+     * Protobuf type {@code google.protobuf.compiler.Version}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:google.protobuf.compiler.Version)
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.VersionOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.internal_static_google_protobuf_compiler_Version_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.internal_static_google_protobuf_compiler_Version_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.class, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        major_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        minor_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        patch_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        suffix_ = "";
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.internal_static_google_protobuf_compiler_Version_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version build() {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version buildPartial() {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version result = new org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.major_ = major_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.minor_ = minor_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.patch_ = patch_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.suffix_ = suffix_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version other) {
+        if (other == org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.getDefaultInstance()) return this;
+        if (other.hasMajor()) {
+          setMajor(other.getMajor());
+        }
+        if (other.hasMinor()) {
+          setMinor(other.getMinor());
+        }
+        if (other.hasPatch()) {
+          setPatch(other.getPatch());
+        }
+        if (other.hasSuffix()) {
+          bitField0_ |= 0x00000008;
+          suffix_ = other.suffix_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private int major_ ;
+      /**
+       * <code>optional int32 major = 1;</code>
+       */
+      public boolean hasMajor() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional int32 major = 1;</code>
+       */
+      public int getMajor() {
+        return major_;
+      }
+      /**
+       * <code>optional int32 major = 1;</code>
+       */
+      public Builder setMajor(int value) {
+        bitField0_ |= 0x00000001;
+        major_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 major = 1;</code>
+       */
+      public Builder clearMajor() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        major_ = 0;
+        onChanged();
+        return this;
+      }
+
+      private int minor_ ;
+      /**
+       * <code>optional int32 minor = 2;</code>
+       */
+      public boolean hasMinor() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional int32 minor = 2;</code>
+       */
+      public int getMinor() {
+        return minor_;
+      }
+      /**
+       * <code>optional int32 minor = 2;</code>
+       */
+      public Builder setMinor(int value) {
+        bitField0_ |= 0x00000002;
+        minor_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 minor = 2;</code>
+       */
+      public Builder clearMinor() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        minor_ = 0;
+        onChanged();
+        return this;
+      }
+
+      private int patch_ ;
+      /**
+       * <code>optional int32 patch = 3;</code>
+       */
+      public boolean hasPatch() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional int32 patch = 3;</code>
+       */
+      public int getPatch() {
+        return patch_;
+      }
+      /**
+       * <code>optional int32 patch = 3;</code>
+       */
+      public Builder setPatch(int value) {
+        bitField0_ |= 0x00000004;
+        patch_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 patch = 3;</code>
+       */
+      public Builder clearPatch() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        patch_ = 0;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object suffix_ = "";
+      /**
+       * <pre>
+       * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+       * be empty for mainline stable releases.
+       * </pre>
+       *
+       * <code>optional string suffix = 4;</code>
+       */
+      public boolean hasSuffix() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <pre>
+       * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+       * be empty for mainline stable releases.
+       * </pre>
+       *
+       * <code>optional string suffix = 4;</code>
+       */
+      public java.lang.String getSuffix() {
+        java.lang.Object ref = suffix_;
+        if (!(ref instanceof java.lang.String)) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            suffix_ = s;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <pre>
+       * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+       * be empty for mainline stable releases.
+       * </pre>
+       *
+       * <code>optional string suffix = 4;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getSuffixBytes() {
+        java.lang.Object ref = suffix_;
+        if (ref instanceof String) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          suffix_ = b;
+          return b;
+        } else {
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <pre>
+       * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+       * be empty for mainline stable releases.
+       * </pre>
+       *
+       * <code>optional string suffix = 4;</code>
+       */
+      public Builder setSuffix(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        suffix_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+       * be empty for mainline stable releases.
+       * </pre>
+       *
+       * <code>optional string suffix = 4;</code>
+       */
+      public Builder clearSuffix() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        suffix_ = getDefaultInstance().getSuffix();
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+       * be empty for mainline stable releases.
+       * </pre>
+       *
+       * <code>optional string suffix = 4;</code>
+       */
+      public Builder setSuffixBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        suffix_ = value;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:google.protobuf.compiler.Version)
+    }
+
+    // @@protoc_insertion_point(class_scope:google.protobuf.compiler.Version)
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<Version>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<Version>() {
+      public Version parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new Version(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<Version> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<Version> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
   public interface CodeGeneratorRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:google.protobuf.compiler.CodeGeneratorRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
@@ -175,6 +1048,31 @@ public final class PluginProtos {
      */
     org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.FileDescriptorProtoOrBuilder getProtoFileOrBuilder(
         int index);
+
+    /**
+     * <pre>
+     * The version number of protocol compiler.
+     * </pre>
+     *
+     * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+     */
+    boolean hasCompilerVersion();
+    /**
+     * <pre>
+     * The version number of protocol compiler.
+     * </pre>
+     *
+     * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version getCompilerVersion();
+    /**
+     * <pre>
+     * The version number of protocol compiler.
+     * </pre>
+     *
+     * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.VersionOrBuilder getCompilerVersionOrBuilder();
   }
   /**
    * <pre>
@@ -240,6 +1138,19 @@ public final class PluginProtos {
               parameter_ = bs;
               break;
             }
+            case 26: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = compilerVersion_.toBuilder();
+              }
+              compilerVersion_ = input.readMessage(org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(compilerVersion_);
+                compilerVersion_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
             case 122: {
               if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
                 protoFile_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.FileDescriptorProto>();
@@ -487,6 +1398,39 @@ public final class PluginProtos {
       return protoFile_.get(index);
     }
 
+    public static final int COMPILER_VERSION_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version compilerVersion_;
+    /**
+     * <pre>
+     * The version number of protocol compiler.
+     * </pre>
+     *
+     * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+     */
+    public boolean hasCompilerVersion() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <pre>
+     * The version number of protocol compiler.
+     * </pre>
+     *
+     * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version getCompilerVersion() {
+      return compilerVersion_ == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.getDefaultInstance() : compilerVersion_;
+    }
+    /**
+     * <pre>
+     * The version number of protocol compiler.
+     * </pre>
+     *
+     * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.VersionOrBuilder getCompilerVersionOrBuilder() {
+      return compilerVersion_ == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.getDefaultInstance() : compilerVersion_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -511,6 +1455,9 @@ public final class PluginProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, parameter_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(3, getCompilerVersion());
+      }
       for (int i = 0; i < protoFile_.size(); i++) {
         output.writeMessage(15, protoFile_.get(i));
       }
@@ -533,6 +1480,10 @@ public final class PluginProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, parameter_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, getCompilerVersion());
+      }
       for (int i = 0; i < protoFile_.size(); i++) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(15, protoFile_.get(i));
@@ -563,6 +1514,11 @@ public final class PluginProtos {
       }
       result = result && getProtoFileList()
           .equals(other.getProtoFileList());
+      result = result && (hasCompilerVersion() == other.hasCompilerVersion());
+      if (hasCompilerVersion()) {
+        result = result && getCompilerVersion()
+            .equals(other.getCompilerVersion());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -573,7 +1529,7 @@ public final class PluginProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getFileToGenerateCount() > 0) {
         hash = (37 * hash) + FILE_TO_GENERATE_FIELD_NUMBER;
         hash = (53 * hash) + getFileToGenerateList().hashCode();
@@ -586,6 +1542,10 @@ public final class PluginProtos {
         hash = (37 * hash) + PROTO_FILE_FIELD_NUMBER;
         hash = (53 * hash) + getProtoFileList().hashCode();
       }
+      if (hasCompilerVersion()) {
+        hash = (37 * hash) + COMPILER_VERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getCompilerVersion().hashCode();
+      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -705,6 +1665,7 @@ public final class PluginProtos {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getProtoFileFieldBuilder();
+          getCompilerVersionFieldBuilder();
         }
       }
       public Builder clear() {
@@ -719,6 +1680,12 @@ public final class PluginProtos {
         } else {
           protoFileBuilder_.clear();
         }
+        if (compilerVersionBuilder_ == null) {
+          compilerVersion_ = null;
+        } else {
+          compilerVersionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -761,6 +1728,14 @@ public final class PluginProtos {
         } else {
           result.protoFile_ = protoFileBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (compilerVersionBuilder_ == null) {
+          result.compilerVersion_ = compilerVersion_;
+        } else {
+          result.compilerVersion_ = compilerVersionBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -844,6 +1819,9 @@ public final class PluginProtos {
             }
           }
         }
+        if (other.hasCompilerVersion()) {
+          mergeCompilerVersion(other.getCompilerVersion());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -1597,6 +2575,160 @@ public final class PluginProtos {
         }
         return protoFileBuilder_;
       }
+
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version compilerVersion_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.VersionOrBuilder> compilerVersionBuilder_;
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public boolean hasCompilerVersion() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version getCompilerVersion() {
+        if (compilerVersionBuilder_ == null) {
+          return compilerVersion_ == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.getDefaultInstance() : compilerVersion_;
+        } else {
+          return compilerVersionBuilder_.getMessage();
+        }
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public Builder setCompilerVersion(org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version value) {
+        if (compilerVersionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          compilerVersion_ = value;
+          onChanged();
+        } else {
+          compilerVersionBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public Builder setCompilerVersion(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder builderForValue) {
+        if (compilerVersionBuilder_ == null) {
+          compilerVersion_ = builderForValue.build();
+          onChanged();
+        } else {
+          compilerVersionBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public Builder mergeCompilerVersion(org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version value) {
+        if (compilerVersionBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              compilerVersion_ != null &&
+              compilerVersion_ != org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.getDefaultInstance()) {
+            compilerVersion_ =
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.newBuilder(compilerVersion_).mergeFrom(value).buildPartial();
+          } else {
+            compilerVersion_ = value;
+          }
+          onChanged();
+        } else {
+          compilerVersionBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public Builder clearCompilerVersion() {
+        if (compilerVersionBuilder_ == null) {
+          compilerVersion_ = null;
+          onChanged();
+        } else {
+          compilerVersionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder getCompilerVersionBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getCompilerVersionFieldBuilder().getBuilder();
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.VersionOrBuilder getCompilerVersionOrBuilder() {
+        if (compilerVersionBuilder_ != null) {
+          return compilerVersionBuilder_.getMessageOrBuilder();
+        } else {
+          return compilerVersion_ == null ?
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.getDefaultInstance() : compilerVersion_;
+        }
+      }
+      /**
+       * <pre>
+       * The version number of protocol compiler.
+       * </pre>
+       *
+       * <code>optional .google.protobuf.compiler.Version compiler_version = 3;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.VersionOrBuilder> 
+          getCompilerVersionFieldBuilder() {
+        if (compilerVersionBuilder_ == null) {
+          compilerVersionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.Version.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.compiler.PluginProtos.VersionOrBuilder>(
+                  getCompilerVersion(),
+                  getParentForChildren(),
+                  isClean());
+          compilerVersion_ = null;
+        }
+        return compilerVersionBuilder_;
+      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -2470,7 +3602,7 @@ public final class PluginProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasName()) {
           hash = (37 * hash) + NAME_FIELD_NUMBER;
           hash = (53 * hash) + getName().hashCode();
@@ -3501,7 +4633,7 @@ public final class PluginProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasError()) {
         hash = (37 * hash) + ERROR_FIELD_NUMBER;
         hash = (53 * hash) + getError().hashCode();
@@ -4204,6 +5336,11 @@ public final class PluginProtos {
   }
 
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_google_protobuf_compiler_Version_descriptor;
+  private static final 
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+      internal_static_google_protobuf_compiler_Version_fieldAccessorTable;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_google_protobuf_compiler_CodeGeneratorRequest_descriptor;
   private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
@@ -4229,16 +5366,19 @@ public final class PluginProtos {
     java.lang.String[] descriptorData = {
       "\n%google/protobuf/compiler/plugin.proto\022" +
       "\030google.protobuf.compiler\032 google/protob" +
-      "uf/descriptor.proto\"}\n\024CodeGeneratorRequ" +
-      "est\022\030\n\020file_to_generate\030\001 \003(\t\022\021\n\tparamet" +
-      "er\030\002 \001(\t\0228\n\nproto_file\030\017 \003(\0132$.google.pr" +
-      "otobuf.FileDescriptorProto\"\252\001\n\025CodeGener" +
-      "atorResponse\022\r\n\005error\030\001 \001(\t\022B\n\004file\030\017 \003(" +
-      "\01324.google.protobuf.compiler.CodeGenerat" +
-      "orResponse.File\032>\n\004File\022\014\n\004name\030\001 \001(\t\022\027\n" +
-      "\017insertion_point\030\002 \001(\t\022\017\n\007content\030\017 \001(\tB",
-      "7\n\034com.google.protobuf.compilerB\014PluginP" +
-      "rotosZ\tplugin_go"
+      "uf/descriptor.proto\"F\n\007Version\022\r\n\005major\030" +
+      "\001 \001(\005\022\r\n\005minor\030\002 \001(\005\022\r\n\005patch\030\003 \001(\005\022\016\n\006s" +
+      "uffix\030\004 \001(\t\"\272\001\n\024CodeGeneratorRequest\022\030\n\020" +
+      "file_to_generate\030\001 \003(\t\022\021\n\tparameter\030\002 \001(" +
+      "\t\0228\n\nproto_file\030\017 \003(\0132$.google.protobuf." +
+      "FileDescriptorProto\022;\n\020compiler_version\030" +
+      "\003 \001(\0132!.google.protobuf.compiler.Version" +
+      "\"\252\001\n\025CodeGeneratorResponse\022\r\n\005error\030\001 \001(",
+      "\t\022B\n\004file\030\017 \003(\01324.google.protobuf.compil" +
+      "er.CodeGeneratorResponse.File\032>\n\004File\022\014\n" +
+      "\004name\030\001 \001(\t\022\027\n\017insertion_point\030\002 \001(\t\022\017\n\007" +
+      "content\030\017 \001(\tB7\n\034com.google.protobuf.com" +
+      "pilerB\014PluginProtosZ\tplugin_go"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -4253,14 +5393,20 @@ public final class PluginProtos {
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.getDescriptor(),
         }, assigner);
-    internal_static_google_protobuf_compiler_CodeGeneratorRequest_descriptor =
+    internal_static_google_protobuf_compiler_Version_descriptor =
       getDescriptor().getMessageTypes().get(0);
+    internal_static_google_protobuf_compiler_Version_fieldAccessorTable = new
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
+        internal_static_google_protobuf_compiler_Version_descriptor,
+        new java.lang.String[] { "Major", "Minor", "Patch", "Suffix", });
+    internal_static_google_protobuf_compiler_CodeGeneratorRequest_descriptor =
+      getDescriptor().getMessageTypes().get(1);
     internal_static_google_protobuf_compiler_CodeGeneratorRequest_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_google_protobuf_compiler_CodeGeneratorRequest_descriptor,
-        new java.lang.String[] { "FileToGenerate", "Parameter", "ProtoFile", });
+        new java.lang.String[] { "FileToGenerate", "Parameter", "ProtoFile", "CompilerVersion", });
     internal_static_google_protobuf_compiler_CodeGeneratorResponse_descriptor =
-      getDescriptor().getMessageTypes().get(1);
+      getDescriptor().getMessageTypes().get(2);
     internal_static_google_protobuf_compiler_CodeGeneratorResponse_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_google_protobuf_compiler_CodeGeneratorResponse_descriptor,

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java
index 2e78905..46e9c01 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java
@@ -210,7 +210,7 @@ public final class TestProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java
index 36a15c9..a30032d 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProtos.java
@@ -130,7 +130,7 @@ public final class TestProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -503,7 +503,7 @@ public final class TestProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -956,7 +956,7 @@ public final class TestProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasMessage()) {
         hash = (37 * hash) + MESSAGE_FIELD_NUMBER;
         hash = (53 * hash) + getMessage().hashCode();
@@ -1507,7 +1507,7 @@ public final class TestProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasMessage()) {
         hash = (37 * hash) + MESSAGE_FIELD_NUMBER;
         hash = (53 * hash) + getMessage().hashCode();
@@ -2026,7 +2026,7 @@ public final class TestProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasMs()) {
         hash = (37 * hash) + MS_FIELD_NUMBER;
         hash = (53 * hash) + getMs();
@@ -2531,7 +2531,7 @@ public final class TestProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasAddr()) {
         hash = (37 * hash) + ADDR_FIELD_NUMBER;
         hash = (53 * hash) + getAddr().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
index b4bd84d..703de38 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
@@ -241,7 +241,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -1077,7 +1077,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegionInfo()) {
         hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getRegionInfo().hashCode();
@@ -1831,7 +1831,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -2528,7 +2528,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getStoreFileCount() > 0) {
         hash = (37 * hash) + STORE_FILE_FIELD_NUMBER;
         hash = (53 * hash) + getStoreFileList().hashCode();
@@ -3017,7 +3017,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3478,7 +3478,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionInfoCount() > 0) {
         hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getRegionInfoList().hashCode();
@@ -4553,7 +4553,7 @@ public final class AdminProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasRegion()) {
           hash = (37 * hash) + REGION_FIELD_NUMBER;
           hash = (53 * hash) + getRegion().hashCode();
@@ -5526,7 +5526,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getOpenInfoCount() > 0) {
         hash = (37 * hash) + OPEN_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getOpenInfoList().hashCode();
@@ -6512,7 +6512,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getOpeningStateCount() > 0) {
         hash = (37 * hash) + OPENING_STATE_FIELD_NUMBER;
         hash = (53 * hash) + openingState_.hashCode();
@@ -7051,7 +7051,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegionInfo()) {
         hash = (37 * hash) + REGIONINFO_FIELD_NUMBER;
         hash = (53 * hash) + getRegionInfo().hashCode();
@@ -7574,7 +7574,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -8229,7 +8229,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -9104,7 +9104,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasClosed()) {
         hash = (37 * hash) + CLOSED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -9624,7 +9624,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionCount() > 0) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegionList().hashCode();
@@ -10344,7 +10344,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasClosed()) {
         hash = (37 * hash) + CLOSED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -10946,7 +10946,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -11716,7 +11716,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLastFlushTime()) {
         hash = (37 * hash) + LAST_FLUSH_TIME_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -12354,7 +12354,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -12934,7 +12934,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -13467,7 +13467,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -14091,7 +14091,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -14727,7 +14727,7 @@ public final class AdminProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasRegion()) {
           hash = (37 * hash) + REGION_FIELD_NUMBER;
           hash = (53 * hash) + getRegion().hashCode();
@@ -15522,7 +15522,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getUpdateInfoCount() > 0) {
         hash = (37 * hash) + UPDATE_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUpdateInfoList().hashCode();
@@ -16232,7 +16232,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasResponse()) {
         hash = (37 * hash) + RESPONSE_FIELD_NUMBER;
         hash = (53 * hash) + getResponse();
@@ -16882,7 +16882,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasKey()) {
         hash = (37 * hash) + KEY_FIELD_NUMBER;
         hash = (53 * hash) + getKey().hashCode();
@@ -17923,7 +17923,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getEntryCount() > 0) {
         hash = (37 * hash) + ENTRY_FIELD_NUMBER;
         hash = (53 * hash) + getEntryList().hashCode();
@@ -18871,7 +18871,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -19244,7 +19244,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -19708,7 +19708,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionToFlushCount() > 0) {
         hash = (37 * hash) + REGION_TO_FLUSH_FIELD_NUMBER;
         hash = (53 * hash) + getRegionToFlushList().hashCode();
@@ -20289,7 +20289,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasReason()) {
         hash = (37 * hash) + REASON_FIELD_NUMBER;
         hash = (53 * hash) + getReason().hashCode();
@@ -20760,7 +20760,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -21133,7 +21133,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -21617,7 +21617,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServerName()) {
         hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getServerName().hashCode();
@@ -22254,7 +22254,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServerInfo()) {
         hash = (37 * hash) + SERVER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getServerInfo().hashCode();
@@ -22777,7 +22777,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -23150,7 +23150,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -23590,7 +23590,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -24200,7 +24200,7 @@ public final class AdminProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionLoadsCount() > 0) {
         hash = (37 * hash) + REGION_LOADS_FIELD_NUMBER;
         hash = (53 * hash) + getRegionLoadsList().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java
index 4cad101..0271777 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java
@@ -334,7 +334,7 @@ public final class BackupProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServerName()) {
         hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getServerName().hashCode();
@@ -1067,7 +1067,7 @@ public final class BackupProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -2434,7 +2434,7 @@ public final class BackupProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBackupId()) {
         hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER;
         hash = (53 * hash) + getBackupId().hashCode();
@@ -4158,7 +4158,7 @@ public final class BackupProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -5690,7 +5690,7 @@ public final class BackupProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBackupId()) {
         hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER;
         hash = (53 * hash) + getBackupId().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java
index 60ca467..e1d7121 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/CellProtos.java
@@ -576,7 +576,7 @@ public final class CellProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRow()) {
         hash = (37 * hash) + ROW_FIELD_NUMBER;
         hash = (53 * hash) + getRow().hashCode();
@@ -1613,7 +1613,7 @@ public final class CellProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRow()) {
         hash = (37 * hash) + ROW_FIELD_NUMBER;
         hash = (53 * hash) + getRow().hashCode();


[46/50] [abbrv] hbase git commit: HBASE-16875 Changed try-with-resources in the docs to recommended way

Posted by sy...@apache.org.
HBASE-16875 Changed try-with-resources in the docs to recommended way

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: c8cd921bededa67b2b0de823005830d750534d93
Parents: c1ac3f7
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sat Mar 4 10:04:02 2017 +0100
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Apr 17 10:59:46 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/architecture.adoc |  7 +++---
 src/main/asciidoc/_chapters/security.adoc     | 29 ++++++++--------------
 2 files changed, 13 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c8cd921b/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 27aebd9..7f9ba07 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -219,10 +219,9 @@ For applications which require high-end multithreaded access (e.g., web-servers
 ----
 // Create a connection to the cluster.
 Configuration conf = HBaseConfiguration.create();
-try (Connection connection = ConnectionFactory.createConnection(conf)) {
-  try (Table table = connection.getTable(TableName.valueOf(tablename)) {
-    // use table as needed, the table returned is lightweight
-  }
+try (Connection connection = ConnectionFactory.createConnection(conf);
+     Table table = connection.getTable(TableName.valueOf(tablename))) {
+  // use table as needed, the table returned is lightweight
 }
 ----
 ====

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8cd921b/src/main/asciidoc/_chapters/security.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/security.adoc b/src/main/asciidoc/_chapters/security.adoc
index 0ed9ba2..ccb5adb 100644
--- a/src/main/asciidoc/_chapters/security.adoc
+++ b/src/main/asciidoc/_chapters/security.adoc
@@ -202,10 +202,9 @@ Set it in the `Configuration` supplied to `Table`:
 Configuration conf = HBaseConfiguration.create();
 Connection connection = ConnectionFactory.createConnection(conf);
 conf.set("hbase.rpc.protection", "privacy");
-try (Connection connection = ConnectionFactory.createConnection(conf)) {
-  try (Table table = connection.getTable(TableName.valueOf(tablename)) {
+try (Connection connection = ConnectionFactory.createConnection(conf);
+     Table table = connection.getTable(TableName.valueOf(tablename))) {
   .... do your stuff
-  }
 }
 ----
 
@@ -1014,24 +1013,16 @@ public static void grantOnTable(final HBaseTestingUtility util, final String use
   SecureTestUtil.updateACLs(util, new Callable<Void>() {
     @Override
     public Void call() throws Exception {
-      Configuration conf = HBaseConfiguration.create();
-      Connection connection = ConnectionFactory.createConnection(conf);
-      try (Connection connection = ConnectionFactory.createConnection(conf)) {
-        try (Table table = connection.getTable(TableName.valueOf(tablename)) {
-          AccessControlLists.ACL_TABLE_NAME);
-          try {
-            BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
-            AccessControlService.BlockingInterface protocol =
-                AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, table, family, qualifier, actions);
-          } finally {
-            acl.close();
-          }
-          return null;
-        }
+      try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration());
+           Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+        BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
+        AccessControlService.BlockingInterface protocol =
+          AccessControlService.newBlockingStub(service);
+        AccessControlUtil.grant(null, protocol, user, table, family, qualifier, false, actions);
       }
+      return null;
     }
-  }
+  });
 }
 ----
 


[18/50] [abbrv] hbase git commit: HBASE-17858 Update refguide about the IS annotation if necessary

Posted by sy...@apache.org.
HBASE-17858 Update refguide about the IS annotation if necessary


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

Branch: refs/heads/hbase-12439
Commit: 17737b2710a2a1271eb791478eb99f7a573ecac1
Parents: 029fa29
Author: zhangduo <zh...@apache.org>
Authored: Fri Mar 31 18:20:45 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Apr 6 09:48:18 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/upgrading.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/17737b27/src/main/asciidoc/_chapters/upgrading.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index df5bbfe..46f637d 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -128,7 +128,7 @@ for warning about incompatible changes). All effort will be made to provide a de
 HBase has a lot of API points, but for the compatibility matrix above, we differentiate between Client API, Limited Private API, and Private API. HBase uses a version of link:https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/Compatibility.html[Hadoop's Interface classification]. HBase's Interface classification classes can be found link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/classification/package-summary.html[here].
 
 * InterfaceAudience: captures the intended audience, possible values are Public (for end users and external projects), LimitedPrivate (for other Projects, Coprocessors or other plugin points), and Private (for internal use).
-* InterfaceStability: describes what types of interface changes are permitted. Possible values are Stable, Evolving, Unstable, and Deprecated.
+* InterfaceStability: describes what types of interface changes are permitted. Possible values are Stable, Evolving, Unstable, and Deprecated. Notice that this annotation is only valid for classes which are marked as IA.LimitedPrivate. The stability of IA.Public classes is only related to the upgrade type(major, minor or patch). And for IA.Private classes, there is no guarantee on the stability between releases. Refer to the Compatibility Matrix above for more details.
 
 [[hbase.client.api]]
 HBase Client API::


[31/50] [abbrv] hbase git commit: HBASE-17905: [hbase-spark] bulkload does not work when table not exist - revert due to misspelling

Posted by sy...@apache.org.
HBASE-17905: [hbase-spark] bulkload does not work when table not exist - revert due to misspelling


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

Branch: refs/heads/hbase-12439
Commit: 02da5a6104d413570472ae4621e44fa7e75c0ca6
Parents: 22f602c
Author: tedyu <yu...@gmail.com>
Authored: Tue Apr 11 17:18:37 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Apr 11 17:18:37 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/spark/BulkLoadPartitioner.scala  | 13 +++++--------
 .../apache/hadoop/hbase/spark/HBaseContext.scala  | 18 +-----------------
 2 files changed, 6 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/02da5a61/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
index 022c933..ab4fc41 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
@@ -33,8 +33,8 @@ import org.apache.spark.Partitioner
 @InterfaceAudience.Public
 class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
   extends Partitioner {
-  // when table not exist, startKeys = Byte[0][]
-  override def numPartitions: Int = if (startKeys.length == 0) 1 else startKeys.length
+
+  override def numPartitions: Int = startKeys.length
 
   override def getPartition(key: Any): Int = {
 
@@ -53,11 +53,8 @@ class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
         case _ =>
           key.asInstanceOf[Array[Byte]]
       }
-    var partition = util.Arrays.binarySearch(startKeys, rowKey, comparator)
-    if (partition < 0)
-      partition = partition * -1 + -2
-    if (partition < 0)
-      partition = 0
-    partition
+    val partition = util.Arrays.binarySearch(startKeys, rowKey, comparator)
+    if (partition < 0) partition * -1 + -2
+    else partition
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/02da5a61/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 8c4e0f4..e2891db 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
@@ -48,7 +48,7 @@ import org.apache.spark.streaming.dstream.DStream
 import java.io._
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod
-import org.apache.hadoop.fs.{Path, FileAlreadyExistsException, FileSystem}
+import org.apache.hadoop.fs.{Path, FileSystem}
 import scala.collection.mutable
 
 /**
@@ -620,17 +620,9 @@ class HBaseContext(@transient sc: SparkContext,
                   compactionExclude: Boolean = false,
                   maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):
   Unit = {
-    val stagingPath = new Path(stagingDir)
-    val fs = stagingPath.getFileSystem(config)
-    if (fs.exists(stagingPath)) {
-      throw new FileAlreadyExistsException("Path " + stagingDir + " already exist")
-    }
     val conn = HBaseConnectionCache.getConnection(config)
     val regionLocator = conn.getRegionLocator(tableName)
     val startKeys = regionLocator.getStartKeys
-    if (startKeys.length == 0) {
-      logInfo("Table " + tableName.toString + " was not found")
-    }
     val defaultCompressionStr = config.get("hfile.compression",
       Compression.Algorithm.NONE.getName)
     val hfileCompression = HFileWriterImpl
@@ -751,17 +743,9 @@ class HBaseContext(@transient sc: SparkContext,
                   compactionExclude: Boolean = false,
                   maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):
   Unit = {
-    val stagingPath = new Path(stagingDir)
-    val fs = stagingPath.getFileSystem(config)
-    if (fs.exists(stagingPath)) {
-      throw new FileAlreadyExistsException("Path " + stagingDir + " already exist")
-    }
     val conn = HBaseConnectionCache.getConnection(config)
     val regionLocator = conn.getRegionLocator(tableName)
     val startKeys = regionLocator.getStartKeys
-    if (startKeys.length == 0) {
-      logInfo("Table " + tableName.toString + " was not found")
-    }
     val defaultCompressionStr = config.get("hfile.compression",
       Compression.Algorithm.NONE.getName)
     val defaultCompression = HFileWriterImpl


[43/50] [abbrv] hbase git commit: HBASE-17903 Corrected the alias for the link of HBASE-6580

Posted by sy...@apache.org.
HBASE-17903 Corrected the alias for the link of HBASE-6580

Signed-off-by: CHIA-PING TSAI <ch...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: 918aa4655c4109159f27b6d78460bd3681c11f06
Parents: 8db9760
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sun Apr 16 17:02:47 2017 +0200
Committer: CHIA-PING TSAI <ch...@gmail.com>
Committed: Mon Apr 17 10:22:25 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/architecture.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/918aa465/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 773d237..27aebd9 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -230,7 +230,7 @@ try (Connection connection = ConnectionFactory.createConnection(conf)) {
 .`HTablePool` is Deprecated
 [WARNING]
 ====
-Previous versions of this guide discussed `HTablePool`, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by link:https://issues.apache.org/jira/browse/HBASE-6580[HBASE-6500], or `HConnection`, which is deprecated in HBase 1.0 by `Connection`.
+Previous versions of this guide discussed `HTablePool`, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by link:https://issues.apache.org/jira/browse/HBASE-6580[HBASE-6580], or `HConnection`, which is deprecated in HBase 1.0 by `Connection`.
 Please use link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Connection.html[Connection] instead.
 ====
 


[11/50] [abbrv] hbase git commit: HBASE-17857 Remove IS annotations from IA.Public classes

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java
index 7e3b350..b70e103 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawString.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -32,7 +31,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see RawStringTerminated
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawString implements DataType<String> {
 
   public static final RawString ASCENDING = new RawString(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java
index d11bead..24a394c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringFixedLength.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 
 /**
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.Order;
  * @see RawString
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawStringFixedLength extends FixedLengthWrapper<String> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java
index 4d89d5b..408b57a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawStringTerminated.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 
 /**
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.util.Order;
  * @see OrderedString
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawStringTerminated extends TerminatedWrapper<String> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java
index 550088a..eea64d9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Struct.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.types;
 import java.util.Iterator;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -78,7 +77,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see DataType#isNullable()
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class Struct implements DataType<Object[]> {
 
   @SuppressWarnings("rawtypes")

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java
index d73a17d..ad4f021 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructBuilder.java
@@ -21,13 +21,11 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A helper for building {@link Struct} instances.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class StructBuilder {
 
   protected final List<DataType<?>> fields = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java
index 11808f4..5793527 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/StructIterator.java
@@ -21,7 +21,6 @@ import java.util.Iterator;
 import java.util.NoSuchElementException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
 /**
@@ -47,7 +46,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class StructIterator implements Iterator<Object> {
 
   protected final PositionedByteRange src;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java
index 7485f88..f05b2ef 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/TerminatedWrapper.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange;
  * {@code skippable} variant.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TerminatedWrapper<T> implements DataType<T> {
 
   protected final DataType<T> wrapped;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java
index 2ffc174..c4e6c6b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union2.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  */
 @SuppressWarnings("unchecked")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class Union2<A, B> implements DataType<Object> {
 
   protected final DataType<A> typeA;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java
index 64afc5b..79c37d0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union3.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  */
 @SuppressWarnings("unchecked")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class Union3<A, B, C> extends Union2<A, B> {
 
   protected final DataType<C> typeC;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java
index 1eb8529..387d8d5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/Union4.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  */
 @SuppressWarnings("unchecked")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class Union4<A, B, C, D> extends Union3<A, B, C> {
 
   protected final DataType<D> typeD;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Base64.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Base64.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Base64.java
index b825c0f..3f021eb 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Base64.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Base64.java
@@ -41,7 +41,6 @@ import java.util.zip.GZIPOutputStream;
 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;
 
 /**
  * Encodes and decodes to and from Base64 notation.
@@ -118,7 +117,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * version: 2.2.1
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Base64 {
 
   /* ******** P U B L I C   F I E L D S ******** */
@@ -1290,7 +1288,6 @@ public class Base64 {
    * @since 1.3
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public static class Base64InputStream extends FilterInputStream {
     private boolean encode;                     // Encoding or decoding
     private int position;                       // Current position in the buffer
@@ -1492,7 +1489,6 @@ public class Base64 {
    * @since 1.3
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public static class Base64OutputStream extends FilterOutputStream {
     private boolean encode;
     private int position;
@@ -1538,7 +1534,6 @@ public class Base64 {
      * @since 1.3
      */
     @InterfaceAudience.Public
-    @InterfaceStability.Stable
     public Base64OutputStream(OutputStream out, int options) {
       super(out);
       this.breakLines = (options & DONT_BREAK_LINES) != DONT_BREAK_LINES;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 4bed97c..ff4c843 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -29,7 +29,6 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.ByteBufferWriter;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -43,7 +42,6 @@ import sun.nio.ch.DirectBuffer;
  */
 @SuppressWarnings("restriction")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class ByteBufferUtils {
 
   // "Compressed integer" serialization helper constants.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
index d547db1..ffb1418 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRange.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Lightweight, reusable class for specifying ranges of byte[]'s.
@@ -60,7 +59,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface ByteRange extends Comparable<ByteRange> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
index 9248b41..672366d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteRangeUtils.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import com.google.common.collect.Lists;
 
@@ -32,7 +31,6 @@ import com.google.common.collect.Lists;
  * Utility methods for working with {@link ByteRange}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ByteRangeUtils {
 
   public static int numEqualPrefixBytes(ByteRange left, ByteRange right, int rightInnerOffset) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
index 626132b..704d97f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
@@ -60,7 +59,6 @@ import com.google.protobuf.ByteString;
  */
 @SuppressWarnings("restriction")
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
     value="EQ_CHECK_FOR_OPERAND_NOT_COMPATIBLE_WITH_THIS",
     justification="It has been like this forever")
@@ -335,7 +333,6 @@ public class Bytes implements Comparable<Bytes> {
    * Byte array comparator class.
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public static class ByteArrayComparator implements RawComparator<byte []> {
     /**
      * Constructor
@@ -363,7 +360,6 @@ public class Bytes implements Comparable<Bytes> {
   // while comparing row keys, start keys etc; but as the largest value for comparing
   // region boundaries for endKeys.
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public static class RowEndKeyComparator extends ByteArrayComparator {
     @Override
     public int compare(byte[] left, byte[] right) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
index 36ca7ad..f5788e9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Counter.java
@@ -22,14 +22,12 @@ import java.util.concurrent.atomic.AtomicLongFieldUpdater;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * High scalable counter. Thread safe.
  * @deprecated use {@link java.util.concurrent.atomic.LongAdder} instead.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @Deprecated
 public class Counter {
   private static final int MAX_CELLS_LENGTH = 1 << 20;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MD5Hash.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MD5Hash.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MD5Hash.java
index 1b2ddb0..29b0bb0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MD5Hash.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MD5Hash.java
@@ -26,14 +26,12 @@ import org.apache.commons.codec.binary.Hex;
 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;
 
 /**
  * Utility class for MD5
  * MD5 hash produces a 128-bit digest.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class MD5Hash {
   private static final Log LOG = LogFactory.getLog(MD5Hash.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Order.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Order.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Order.java
index 9cbbe7e..14a08d3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Order.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Order.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Used to describe or modify the lexicographical sort order of a
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * by replacing each byte with its 1's compliment.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum Order {
 
   ASCENDING {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
index a167562..3e4bc6c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
@@ -27,7 +27,6 @@ import java.math.RoundingMode;
 import java.nio.charset.Charset;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -267,7 +266,6 @@ import com.google.common.annotations.VisibleForTesting;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedBytes {
 
   /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Pair.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Pair.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Pair.java
index 719d1ee..f42f63b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Pair.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Pair.java
@@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.util;
 import java.io.Serializable;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A generic class for pairs.
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * @param <T2>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Pair<T1, T2> implements Serializable
 {
   private static final long serialVersionUID = -3986244606585552569L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java
index b9d9da9..3d520a8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A generic, immutable class for pairs of objects both of type <code>T</code>.
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * @see Pair if Types differ.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class PairOfSameType<T> implements Iterable<T> {
   private final T first;
   private final T second;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
index 3c47d86..3643fd6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/PositionedByteRange.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.util;
 
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * <p>
@@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface PositionedByteRange extends ByteRange {
 
   // net new API is here.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java
index c14f1e2..2962c28 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReadOnlyByteRangeException.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Exception thrown when a read only byte range is modified
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ReadOnlyByteRangeException extends UnsupportedOperationException {
   public ReadOnlyByteRangeException() {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
index 4d5e5b5..3b157f8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleByteRange.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  *  A read only version of the {@link ByteRange}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class SimpleByteRange extends AbstractByteRange {
   public SimpleByteRange() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java
index d325649..3b32be0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimpleMutableByteRange.java
@@ -51,13 +51,11 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A basic mutable {@link ByteRange} implementation.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class SimpleMutableByteRange extends AbstractByteRange {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
index 2265f24..2f4da00 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedByteRange.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Extends the basic {@link SimpleMutableByteRange} implementation with position
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * {@link #equals(Object)}. {@code Position} is retained by copy operations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @edu.umd.cs.findbugs.annotations.SuppressWarnings("EQ_DOESNT_OVERRIDE_EQUALS")
 public class SimplePositionedByteRange extends AbstractPositionedByteRange {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java
index b3614bb..5fe53db 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/SimplePositionedMutableByteRange.java
@@ -51,7 +51,6 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Extends the basic {@link AbstractPositionedByteRange} implementation with
@@ -61,7 +60,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * {@link #equals(Object)}. {@code Position} is retained by copy operations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @edu.umd.cs.findbugs.annotations.SuppressWarnings("EQ_DOESNT_OVERRIDE_EQUALS")
 public class SimplePositionedMutableByteRange extends AbstractPositionedByteRange {
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
index 908689c..44bb89f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
@@ -25,13 +25,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Version;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This class finds the Version information for HBase.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class VersionInfo {
   private static final Log LOG = LogFactory.getLog(VersionInfo.class.getName());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
index fcad895..e191046 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
@@ -28,14 +28,12 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Common helpers for testing HBase that do not depend on specific server/etc. things.
  * {@see org.apache.hadoop.hbase.HBaseTestingUtility}
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public class HBaseCommonTestingUtility {
   protected static final Log LOG = LogFactory.getLog(HBaseCommonTestingUtility.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java
----------------------------------------------------------------------
diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java
index 810778b..7760bdc 100644
--- a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java
+++ b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java
@@ -837,4 +837,4 @@ public class AggregationClient implements Closeable {
     }
     return bytes;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
----------------------------------------------------------------------
diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
index 30f3d30..82764b5 100644
--- a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
+++ b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AsyncAggregationClient.java
@@ -33,7 +33,6 @@ import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.RawAsyncTable;
 import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallback;
 import org.apache.hadoop.hbase.client.RawScanResultConsumer;
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * summing/processing the individual results obtained from the AggregateService for each region.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public class AsyncAggregationClient {
 
   private static abstract class AbstractAggregationCallback<T>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
index ea8bc9c..3326f2f 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.rest;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Common constants for org.apache.hadoop.hbase.rest
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public interface Constants {
   // All constants in a public interface are 'public static final'
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
index c9cf49a..4a89aa8 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
@@ -32,7 +32,6 @@ import java.util.concurrent.ConcurrentHashMap;
 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.http.Header;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
@@ -53,7 +52,6 @@ import org.apache.http.util.EntityUtils;
  * semantics for interacting with the REST gateway.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Client {
   public static final Header[] EMPTY_HEADER_ARRAY = new Header[0];
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
index 549a1b2..6189ed6 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java
@@ -24,14 +24,12 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A list of 'host:port' addresses of HTTP servers operating as a single
  * entity, for example multiple redundant web service gateways.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Cluster {
   protected List<String> nodes = 
     Collections.synchronizedList(new ArrayList<String>());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
index de1e23f..55eeba5 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
@@ -31,7 +31,6 @@ import javax.xml.stream.XMLStreamException;
 import javax.xml.stream.XMLStreamReader;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.rest.model.VersionModel;
 import org.apache.hadoop.hbase.util.Bytes;
 
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RemoteAdmin {
 
   final Client client;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/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 9cc3198..5012a5a 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
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 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.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -79,7 +78,6 @@ import com.google.protobuf.ServiceException;
  * HTable interface to remote tables accessed via REST gateway
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RemoteHTable implements Table {
 
   private static final Log LOG = LogFactory.getLog(RemoteHTable.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
index 8998e57..86a92bf 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
@@ -25,7 +25,6 @@ import java.io.InputStream;
 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.http.Header;
 import org.apache.http.HttpResponse;
 
@@ -33,7 +32,6 @@ import org.apache.http.HttpResponse;
  * The HTTP result code, response headers, and body of a HTTP response.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Response {
   private static final Log LOG = LogFactory.getLog(Response.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java
index 7224383..013da3f 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.conf.Configuration;
  * attempt as a bad request.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RestCsrfPreventionFilter implements Filter {
 
   private static final Log LOG =

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
index 255ca31..b04e685 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
@@ -27,7 +27,6 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
@@ -58,7 +57,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
  *
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class LocalHBaseCluster {
   private static final Log LOG = LogFactory.getLog(LocalHBaseCluster.class);
   private final List<JVMClusterUtil.MasterThread> masterThreads = new CopyOnWriteArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
index 49a718c..42d40eb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
@@ -33,7 +33,6 @@ 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.classification.InterfaceStability;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.FSUtils;
 
@@ -65,7 +64,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
  * @see org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TableSnapshotScanner extends AbstractClientScanner {
 
   private static final Log LOG = LogFactory.getLog(TableSnapshotScanner.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java
index c141c3e..e1dc7f9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java
@@ -28,7 +28,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
@@ -81,7 +80,6 @@ import com.google.common.annotations.VisibleForTesting;
  * </code>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class EntityLock {
   private static final Log LOG = LogFactory.getLog(EntityLock.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignException.java
index a00ccd9..bff7e8f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignException.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/ForeignException.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage;
@@ -41,7 +40,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.Sta
  * ProxyThrowables are generated are by this class's {@link #deserialize(byte[])} method.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @SuppressWarnings("serial")
 public class ForeignException extends IOException {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutException.java
index 746c59b..bd2adf3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutException.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/errorhandling/TimeoutException.java
@@ -18,14 +18,12 @@
 package org.apache.hadoop.hbase.errorhandling;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Exception for timeout of a task.
  * @see TimeoutExceptionInjector
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @SuppressWarnings("serial")
 public class TimeoutException extends Exception {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
index e1ca999..a534224 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
@@ -39,7 +38,6 @@ import org.apache.hadoop.mapred.Reporter;
  * Extract grouping columns from input record
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class GroupingTableMap
 extends MapReduceBase
 implements TableMap<ImmutableBytesWritable,Result> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java
index be131e8..0011a60 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java
@@ -25,7 +25,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -44,7 +43,6 @@ import org.apache.hadoop.mapred.Partitioner;
  * @param <V2>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class HRegionPartitioner<K2,V2>
 implements Partitioner<ImmutableBytesWritable, V2> {
   private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class);
@@ -95,4 +93,4 @@ implements Partitioner<ImmutableBytesWritable, V2> {
     // if above fails to find start key that match we need to return something
     return 0;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java
index 2f5a9b8..dfacff9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapred;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.mapred.JobConf;
@@ -33,7 +32,6 @@ import org.apache.hadoop.mapred.Reporter;
  * Pass the given key and record as-is to reduce
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class IdentityTableMap
 extends MapReduceBase
 implements TableMap<ImmutableBytesWritable, Result> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java
index 8d4d301..9c2e604 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java
@@ -24,7 +24,6 @@ import java.util.Iterator;
 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.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -35,7 +34,6 @@ import org.apache.hadoop.mapred.Reporter;
  * Write to table each key, record pair
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class IdentityTableReduce
 extends MapReduceBase
 implements TableReduce<ImmutableBytesWritable, Put> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/MultiTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/MultiTableSnapshotInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/MultiTableSnapshotInputFormat.java
index ab27edd..3e121fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/MultiTableSnapshotInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/MultiTableSnapshotInputFormat.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapred;
 import org.apache.hadoop.conf.Configuration;
 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.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -83,7 +82,6 @@ import java.util.Map;
  * @see org.apache.hadoop.hbase.client.TableSnapshotScanner
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MultiTableSnapshotInputFormat extends TableSnapshotInputFormat
     implements InputFormat<ImmutableBytesWritable, Result> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
index fd9a60c..43560fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapred;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -41,7 +40,6 @@ import org.apache.hadoop.util.ToolRunner;
  * Uses a org.apache.hadoop.mapred.lib.IdentityReducer
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RowCounter extends Configured implements Tool {
   // Name of this 'program'
   static final String NAME = "rowcounter";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
index 29763f3..208849a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
@@ -23,7 +23,6 @@ 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.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
@@ -38,7 +37,6 @@ import org.apache.hadoop.util.StringUtils;
  * Convert HBase tabular data into a format that is consumable by Map/Reduce.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableInputFormat extends TableInputFormatBase implements
     JobConfigurable {
   private static final Log LOG = LogFactory.getLog(TableInputFormat.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
index ccd8417..c65810f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
@@ -24,7 +24,6 @@ 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.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
@@ -78,7 +77,6 @@ import org.apache.hadoop.mapred.Reporter;
  */
 
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class TableInputFormatBase
 implements InputFormat<ImmutableBytesWritable, Result> {
   private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMap.java
index 3a57bc3..a9f1e61 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMap.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.mapred;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableComparable;
@@ -33,7 +32,6 @@ import org.apache.hadoop.mapred.Mapper;
  * @param <V> Writable value class
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public interface TableMap<K extends WritableComparable<? super K>, V>
 extends Mapper<ImmutableBytesWritable, Result, K, V> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
index 73340c4..63ec418 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
@@ -49,7 +48,6 @@ import java.util.Map;
  * Utility for {@link TableMap} and {@link TableReduce}
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public class TableMapReduceUtil {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
index 3fe5a90..eb4b66f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 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.BufferedMutator;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -41,7 +40,6 @@ import org.apache.hadoop.util.Progressable;
  * Convert Map/Reduce output and write it to an HBase table
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableOutputFormat extends FileOutputFormat<ImmutableBytesWritable, Put> {
 
   /** JobConf parameter that specifies the output table */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
index 281d13e..cecef7d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapred;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -33,7 +32,6 @@ import org.apache.hadoop.mapred.RecordReader;
  * Iterate over an HBase table data, return (Text, RowResult) pairs
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableRecordReader
 implements RecordReader<ImmutableBytesWritable, Result> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
index 2ace8e2..f6b79c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
@@ -23,7 +23,6 @@ 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.conf.Configuration;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -43,7 +42,6 @@ import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_RO
  * Iterate over an HBase table data, return (Text, RowResult) pairs
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableRecordReaderImpl {
   private static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableReduce.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableReduce.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableReduce.java
index 2ca7cf8..91fb4a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableReduce.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableReduce.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.mapred;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.WritableComparable;
@@ -32,7 +31,6 @@ import org.apache.hadoop.mapred.Reducer;
  * @param <V> value class
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 @SuppressWarnings("unchecked")
 public interface TableReduce<K extends WritableComparable, V>
 extends Reducer<K, V, ImmutableBytesWritable, Put> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSnapshotInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSnapshotInputFormat.java
index a5c62b2..d7b49ff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSnapshotInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSnapshotInputFormat.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.fs.Path;
 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;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -45,7 +44,6 @@ import java.util.List;
  * @see org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TableSnapshotInputFormat implements InputFormat<ImmutableBytesWritable, Result> {
 
   public static class TableSnapshotRegionSplit implements InputSplit {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java
index 237fe47..0784e5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -34,7 +33,6 @@ import org.apache.hadoop.mapred.InputSplit;
  * A table split corresponds to a key range [low, high)
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableSplit implements InputSplit, Comparable<TableSplit> {
   private TableName m_tableName;
   private byte [] m_startRow;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java
index d39efaa..9a8911e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java
@@ -24,7 +24,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
@@ -71,7 +70,6 @@ import com.google.common.base.Preconditions;
  * start time and/or end time to limit the count to a time range.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class CellCounter extends Configured implements Tool {
   private static final Log LOG =
     LogFactory.getLog(CellCounter.class.getName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java
index 001f64d..1d4d37b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCreator.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
@@ -32,7 +31,6 @@ import org.apache.hadoop.util.ReflectionUtils;
  * Facade to create Cells for HFileOutputFormat. The created Cells are of <code>Put</code> type.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CellCreator {
 
   public static final String VISIBILITY_EXP_RESOLVER_CLASS =

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
index 8f0504a..c0d809b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 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.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -48,7 +47,6 @@ import org.apache.hadoop.util.ToolRunner;
  * of the region server implementation if different from the local cluster.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class CopyTable extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(CopyTable.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Export.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Export.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Export.java
index 3a3988e..4c01528 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Export.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Export.java
@@ -23,7 +23,6 @@ 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.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
@@ -50,7 +49,6 @@ import org.apache.hadoop.util.ToolRunner;
 * back in again.
 */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Export extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(Export.class);
   final static String NAME = "export";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/GroupingTableMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/GroupingTableMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/GroupingTableMapper.java
index 44e43c8..dc30c6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/GroupingTableMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/GroupingTableMapper.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -38,7 +37,6 @@ import org.apache.hadoop.mapreduce.Job;
  * Extract grouping columns from input record.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class GroupingTableMapper
 extends TableMapper<ImmutableBytesWritable,Result> implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index 1ce5f60..5b1f13c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
@@ -92,7 +91,6 @@ import com.google.common.annotations.VisibleForTesting;
  * using {@link #configureIncrementalLoad(Job, HTableDescriptor, RegionLocator, Class)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HFileOutputFormat2
     extends FileOutputFormat<ImmutableBytesWritable, Cell> {
   private static final Log LOG = LogFactory.getLog(HFileOutputFormat2.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
index 4c75b56..3475a48 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java
@@ -23,7 +23,6 @@ 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.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -50,7 +49,6 @@ import org.apache.hadoop.mapreduce.Partitioner;
  * @param <VALUE>  The type of the value.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class HRegionPartitioner<KEY, VALUE>
 extends Partitioner<ImmutableBytesWritable, VALUE>
 implements Configurable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.java
index fdf351e..7103ef8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -31,7 +30,6 @@ import org.apache.hadoop.mapreduce.Job;
  * Pass the given key and record as-is to the reduce phase.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class IdentityTableMapper
 extends TableMapper<ImmutableBytesWritable, Result> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.java
index ec3192e..5289f46 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.java
@@ -23,7 +23,6 @@ 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.hbase.client.Mutation;
 import org.apache.hadoop.io.Writable;
 
@@ -50,7 +49,6 @@ import org.apache.hadoop.io.Writable;
  * row and columns implicitly.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class IdentityTableReducer
 extends TableReducer<Writable, Mutation, Writable> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index d1beb8d..3c72c2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 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;
@@ -84,7 +83,6 @@ import org.apache.zookeeper.KeeperException;
  * Import data written by {@link Export}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Import extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(Import.class);
   final static String NAME = "import";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
index a379d53..aa7b129 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 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;
@@ -74,7 +73,6 @@ import com.google.common.collect.Lists;
  * @see ImportTsv#usage(String)
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ImportTsv extends Configured implements Tool {
 
   protected static final Log LOG = LogFactory.getLog(ImportTsv.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java
index e85ef58..241608b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSerialization.java
@@ -26,13 +26,11 @@ import java.io.OutputStream;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.Serialization;
 import org.apache.hadoop.io.serializer.Serializer;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class KeyValueSerialization implements Serialization<KeyValue> {
   @Override
   public boolean accept(Class<?> c) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java
index d37ab94..5c7ace2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/KeyValueSortReducer.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce;
 import java.util.TreeSet;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -35,7 +34,6 @@ import org.apache.hadoop.mapreduce.Reducer;
  * @see HFileOutputFormat2
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class KeyValueSortReducer extends Reducer<ImmutableBytesWritable, KeyValue, ImmutableBytesWritable, KeyValue> {
   protected void reduce(ImmutableBytesWritable row, java.lang.Iterable<KeyValue> kvs,
       org.apache.hadoop.mapreduce.Reducer<ImmutableBytesWritable, KeyValue, ImmutableBytesWritable, KeyValue>.Context context)

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 80dfd66..19daeed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.backup.impl.BackupManager;
 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.Admin;
 import org.apache.hadoop.hbase.client.ClientServiceCallable;
 import org.apache.hadoop.hbase.client.Connection;
@@ -108,7 +107,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * Tool to load the output of HFileOutputFormat into an existing table.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class LoadIncrementalHFiles extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
   private boolean initalized = false;


[39/50] [abbrv] hbase git commit: HBASE-17888: Added generic methods for updating metrics on submit and finish of a procedure execution

Posted by sy...@apache.org.
HBASE-17888: Added generic methods for updating metrics on submit and finish of a procedure execution

Signed-off-by: Michael 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/c8461456
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c8461456
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c8461456

Branch: refs/heads/hbase-12439
Commit: c8461456d0ae81b90d67d36e1e077ae1d01102e5
Parents: e2a7461
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Mon Apr 10 15:32:43 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Apr 14 11:51:08 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   2 +-
 .../org/apache/hadoop/hbase/ProcedureInfo.java  |  20 +-
 .../master/MetricsAssignmentManagerSource.java  |   9 +-
 .../MetricsAssignmentManagerSourceImpl.java     |   9 +-
 .../hadoop/hbase/procedure2/Procedure.java      |  41 +-
 .../hbase/procedure2/ProcedureExecutor.java     |  11 +
 .../hadoop/hbase/procedure2/ProcedureUtil.java  |  10 +-
 .../hbase/procedure2/TestProcedureMetrics.java  | 254 ++++++++++
 .../procedure2/TestStateMachineProcedure.java   |   1 -
 .../shaded/protobuf/generated/MasterProtos.java | 490 +++++++++----------
 .../protobuf/generated/ProcedureProtos.java     | 146 +++---
 .../src/main/protobuf/Master.proto              |   2 +-
 .../src/main/protobuf/Procedure.proto           |   2 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   4 +-
 .../master/procedure/ServerCrashProcedure.java  |   2 +-
 .../hbase-webapps/master/procedures.jsp         |   2 +-
 .../main/ruby/shell/commands/list_procedures.rb |   6 +-
 17 files changed, 652 insertions(+), 359 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/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 155a272..cadd6cc 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
@@ -2114,7 +2114,7 @@ public class HBaseAdmin implements Admin {
         procedureState, procProto.hasParentId() ? procProto.getParentId() : -1, nonceKey,
             procProto.hasException()?
                 ForeignExceptionUtil.toIOException(procProto.getException()): null,
-            procProto.getLastUpdate(), procProto.getStartTime(),
+            procProto.getLastUpdate(), procProto.getSubmittedTime(),
             procProto.hasResult()? procProto.getResult().toByteArray() : null);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
index bb8bb08..6104c22 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
@@ -39,7 +39,7 @@ public class ProcedureInfo implements Cloneable {
   private final NonceKey nonceKey;
   private final IOException exception;
   private final long lastUpdate;
-  private final long startTime;
+  private final long submittedTime;
   private final byte[] result;
 
   private long clientAckTime = -1;
@@ -54,7 +54,7 @@ public class ProcedureInfo implements Cloneable {
       final NonceKey nonceKey,
       final IOException exception,
       final long lastUpdate,
-      final long startTime,
+      final long submittedTime,
       final byte[] result) {
     this.procId = procId;
     this.procName = procName;
@@ -63,7 +63,7 @@ public class ProcedureInfo implements Cloneable {
     this.parentId = parentId;
     this.nonceKey = nonceKey;
     this.lastUpdate = lastUpdate;
-    this.startTime = startTime;
+    this.submittedTime = submittedTime;
 
     // If the procedure is completed, we should treat exception and result differently
     this.exception = exception;
@@ -74,7 +74,7 @@ public class ProcedureInfo implements Cloneable {
       justification="Intentional; calling super class clone doesn't make sense here.")
   public ProcedureInfo clone() {
     return new ProcedureInfo(procId, procName, procOwner, procState, parentId, nonceKey,
-      exception, lastUpdate, startTime, result);
+      exception, lastUpdate, submittedTime, result);
   }
 
   @Override
@@ -96,10 +96,10 @@ public class ProcedureInfo implements Cloneable {
     sb.append(procState);
 
     long now = EnvironmentEdgeManager.currentTime();
-    sb.append(", startTime=");
-    sb.append(StringUtils.formatTime(now - startTime));
+    sb.append(", submittedTime=");
+    sb.append(StringUtils.formatTime(now - submittedTime));
     sb.append(" ago, lastUpdate=");
-    sb.append(StringUtils.formatTime(now - startTime));
+    sb.append(StringUtils.formatTime(now - submittedTime));
     sb.append(" ago");
 
     if (isFailed()) {
@@ -168,8 +168,8 @@ public class ProcedureInfo implements Cloneable {
     return result;
   }
 
-  public long getStartTime() {
-    return startTime;
+  public long getSubmittedTime() {
+    return submittedTime;
   }
 
   public long getLastUpdate() {
@@ -177,7 +177,7 @@ public class ProcedureInfo implements Cloneable {
   }
 
   public long executionTime() {
-    return lastUpdate - startTime;
+    return lastUpdate - submittedTime;
   }
 
   @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
index f6c9cb8..fa7bbec 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
@@ -40,7 +40,7 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   /**
    * Description
    */
-  String METRICS_DESCRIPTION = "Metrics about HBase master assingment manager.";
+  String METRICS_DESCRIPTION = "Metrics about HBase master assignment manager.";
 
   String RIT_COUNT_NAME = "ritCount";
   String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
@@ -49,6 +49,13 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   String ASSIGN_TIME_NAME = "assign";
   String BULK_ASSIGN_TIME_NAME = "bulkAssign";
 
+  String RIT_COUNT_DESC = "Current number of Regions In Transition (Gauge).";
+  String RIT_COUNT_OVER_THRESHOLD_DESC =
+      "Current number of Regions In Transition over threshold time (Gauge).";
+  String RIT_OLDEST_AGE_DESC = "Timestamp in milliseconds of the oldest Region In Transition (Gauge).";
+  String RIT_DURATION_DESC =
+      "Total durations in milliseconds for all Regions in Transition (Histogram).";
+
   void updateAssignmentTime(long time);
 
   void updateBulkAssignTime(long time);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/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 ab504f5..faae044 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
@@ -46,12 +46,13 @@ public class MetricsAssignmentManagerSourceImpl
   }
 
   public void init() {
-    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
-    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
-    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
+    ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, RIT_COUNT_DESC, 0l);
+    ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME,
+        RIT_COUNT_OVER_THRESHOLD_DESC,0l);
+    ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, RIT_OLDEST_AGE_DESC, 0l);
     assignTimeHisto = metricsRegistry.newTimeHistogram(ASSIGN_TIME_NAME);
     bulkAssignTimeHisto = metricsRegistry.newTimeHistogram(BULK_ASSIGN_TIME_NAME);
-    ritDurationHisto = metricsRegistry.newTimeHistogram(RIT_DURATION_NAME);
+    ritDurationHisto = metricsRegistry.newTimeHistogram(RIT_DURATION_NAME, RIT_DURATION_DESC);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 761ab3a..591c0d0 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -38,7 +38,7 @@ import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Base Procedure class responsible to handle the Procedure Metadata
- * e.g. state, startTime, lastUpdate, stack-indexes, ...
+ * e.g. state, submittedTime, lastUpdate, stack-indexes, ...
  *
  * execute() is called each time the procedure is executed.
  * it may be called multiple times in case of failure and restart, so the
@@ -73,7 +73,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   private long parentProcId = NO_PROC_ID;
   private long rootProcId = NO_PROC_ID;
   private long procId = NO_PROC_ID;
-  private long startTime;
+  private long submittedTime;
 
   // runtime state, updated every operation
   private ProcedureState state = ProcedureState.INITIALIZING;
@@ -240,6 +240,27 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
     return true;
   }
 
+  /**
+   * This function will be called just when procedure is submitted for execution. Override this
+   * method to update the metrics at the beginning of the procedure
+   */
+  protected void updateMetricsOnSubmit(final TEnvironment env) {}
+
+  /**
+   * This function will be called just after procedure execution is finished. Override this method
+   * to update metrics at the end of the procedure
+   *
+   * TODO: As any of the sub-procedures on failure rolls back all procedures in the stack,
+   * including successfully finished siblings, this function may get called twice in certain
+   * cases for certain procedures. Explore further if this can be called once.
+   *
+   * @param env
+   * @param runtime - Runtime of the procedure in milliseconds
+   * @param success - true if procedure is completed successfully
+   */
+  protected void updateMetricsOnFinish(final TEnvironment env, final long runtime,
+                                       boolean success) {}
+
   @Override
   public String toString() {
     // Return the simple String presentation of the procedure.
@@ -287,8 +308,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   public String toStringDetails() {
     final StringBuilder sb = toStringSimpleSB();
 
-    sb.append(" startTime=");
-    sb.append(getStartTime());
+    sb.append(" submittedTime=");
+    sb.append(getSubmittedTime());
 
     sb.append(" lastUpdate=");
     sb.append(getLastUpdate());
@@ -353,8 +374,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
     return nonceKey;
   }
 
-  public long getStartTime() {
-    return startTime;
+  public long getSubmittedTime() {
+    return submittedTime;
   }
 
   public String getOwner() {
@@ -372,7 +393,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   @InterfaceAudience.Private
   protected void setProcId(final long procId) {
     this.procId = procId;
-    this.startTime = EnvironmentEdgeManager.currentTime();
+    this.submittedTime = EnvironmentEdgeManager.currentTime();
     setState(ProcedureState.RUNNABLE);
   }
 
@@ -414,8 +435,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
    * the creation/deserialization.
    */
   @InterfaceAudience.Private
-  protected void setStartTime(final long startTime) {
-    this.startTime = startTime;
+  protected void setSubmittedTime(final long submittedTime) {
+    this.submittedTime = submittedTime;
   }
 
   // ==========================================================================
@@ -478,7 +499,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
    * @return the time elapsed between the last update and the start time of the procedure.
    */
   public long elapsedTime() {
-    return getLastUpdate() - getStartTime();
+    return getLastUpdate() - getSubmittedTime();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 43f5839..1bb6118 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -831,6 +831,9 @@ public class ProcedureExecutor<TEnvironment> {
   private long pushProcedure(final Procedure proc) {
     final long currentProcId = proc.getProcId();
 
+    // Update metrics on start of a procedure
+    proc.updateMetricsOnSubmit(getEnvironment());
+
     // Create the rollback stack for the procedure
     RootProcedureState stack = new RootProcedureState();
     rollbackStack.put(currentProcId, stack);
@@ -1145,6 +1148,9 @@ public class ProcedureExecutor<TEnvironment> {
       }
 
       if (proc.isSuccess()) {
+        // update metrics on finishing the procedure
+        proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), true);
+
         if (LOG.isDebugEnabled()) {
           LOG.debug("Finished " + proc + " in " + StringUtils.humanTimeDiff(proc.elapsedTime()));
         }
@@ -1276,6 +1282,10 @@ public class ProcedureExecutor<TEnvironment> {
 
     if (proc.removeStackIndex()) {
       proc.setState(ProcedureState.ROLLEDBACK);
+
+      // update metrics on finishing the procedure (fail)
+      proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), false);
+
       if (proc.hasParent()) {
         store.delete(proc.getProcId());
         procedures.remove(proc.getProcId());
@@ -1444,6 +1454,7 @@ public class ProcedureExecutor<TEnvironment> {
   private void submitChildrenProcedures(final Procedure[] subprocs) {
     for (int i = 0; i < subprocs.length; ++i) {
       final Procedure subproc = subprocs[i];
+      subproc.updateMetricsOnSubmit(getEnvironment());
       assert !procedures.containsKey(subproc.getProcId());
       procedures.put(subproc.getProcId(), subproc);
       scheduler.addFront(subproc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
index 372dcac..b4222c7 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
@@ -97,7 +97,7 @@ public final class ProcedureUtil {
       .setClassName(proc.getClass().getName())
       .setProcId(proc.getProcId())
       .setState(proc.getState())
-      .setStartTime(proc.getStartTime())
+      .setSubmittedTime(proc.getSubmittedTime())
       .setLastUpdate(proc.getLastUpdate());
 
     if (proc.hasParent()) {
@@ -164,7 +164,7 @@ public final class ProcedureUtil {
     // set fields
     proc.setProcId(proto.getProcId());
     proc.setState(proto.getState());
-    proc.setStartTime(proto.getStartTime());
+    proc.setSubmittedTime(proto.getSubmittedTime());
     proc.setLastUpdate(proto.getLastUpdate());
 
     if (proto.hasParentId()) {
@@ -217,7 +217,7 @@ public final class ProcedureUtil {
 
     builder.setClassName(procInfo.getProcName());
     builder.setProcId(procInfo.getProcId());
-    builder.setStartTime(procInfo.getStartTime());
+    builder.setSubmittedTime(procInfo.getSubmittedTime());
     builder.setState(ProcedureProtos.ProcedureState.valueOf(procInfo.getProcState().name()));
     builder.setLastUpdate(procInfo.getLastUpdate());
 
@@ -257,7 +257,7 @@ public final class ProcedureUtil {
         procProto.hasParentId() ? procProto.getParentId() : -1, nonceKey,
         procProto.hasException() ?
           ForeignExceptionUtil.toIOException(procProto.getException()) : null,
-        procProto.getLastUpdate(), procProto.getStartTime(),
+        procProto.getLastUpdate(), procProto.getSubmittedTime(),
         procProto.hasResult() ? procProto.getResult().toByteArray() : null);
   }
 
@@ -279,6 +279,6 @@ public final class ProcedureUtil {
         convertToProcedureState(proc.getState()),
         proc.hasParent() ? proc.getParentProcId() : -1, nonceKey,
         exception != null ? exception.unwrapRemoteIOException() : null,
-        proc.getLastUpdate(), proc.getStartTime(), proc.getResult());
+        proc.getLastUpdate(), proc.getSubmittedTime(), proc.getResult());
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java
new file mode 100644
index 0000000..0a57efa
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureMetrics.java
@@ -0,0 +1,254 @@
+/**
+ * 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.procedure2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestProcedureMetrics {
+  private static final Log LOG = LogFactory.getLog(TestProcedureMetrics.class);
+
+  private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
+
+  private TestProcEnv procEnv;
+  private static ProcedureExecutor<TestProcEnv> procExecutor;
+  private ProcedureStore procStore;
+
+  private HBaseCommonTestingUtility htu;
+  private FileSystem fs;
+  private Path testDir;
+  private Path logDir;
+
+  private static int beginCount = 0;
+  private static int successCount = 0;
+  private static int failedCount = 0;
+
+
+  @Before
+  public void setUp() throws IOException {
+    htu = new HBaseCommonTestingUtility();
+    testDir = htu.getDataTestDir();
+    fs = testDir.getFileSystem(htu.getConfiguration());
+    assertTrue(testDir.depth() > 1);
+
+    logDir = new Path(testDir, "proc-logs");
+    procEnv = new TestProcEnv();
+    procStore = ProcedureTestingUtility.createStore(htu.getConfiguration(), fs, logDir);
+    procExecutor = new ProcedureExecutor<TestProcEnv>(htu.getConfiguration(), procEnv, procStore);
+    procExecutor.testing = new ProcedureExecutor.Testing();
+    procStore.start(PROCEDURE_EXECUTOR_SLOTS);
+    procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    procExecutor.stop();
+    procStore.stop(false);
+    fs.delete(logDir, true);
+  }
+
+  @Test
+  public void testMetricForSimpleProcedure() throws Exception {
+    // procedure that executes successfully
+    ProcedureMetrics proc = new ProcedureMetrics(true);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    successCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricsForFailedProcedure() throws Exception {
+    // procedure that fails
+    ProcedureMetrics proc = new ProcedureMetrics(false);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    failedCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricForYieldProcedure() throws Exception {
+    // procedure that yields
+    ProcedureMetrics proc = new ProcedureMetrics(true, true);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    successCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricForFailedYiledProcedure() {
+    // procedure that yields and fails
+    ProcedureMetrics proc = new ProcedureMetrics(false, true);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount++;
+    failedCount++;
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  @Test
+  public void testMetricForProcedureWithChildren() throws Exception {
+    // Procedure that yileds with one of the sub-procedures that fail
+    int subProcCount = 10;
+    int failChildIndex = 2;
+    int yiledChildIndex = -1;
+    ProcedureMetrics[] subprocs = new ProcedureMetrics[subProcCount];
+    for (int i = 0; i < subProcCount; ++i) {
+      subprocs[i] = new ProcedureMetrics(failChildIndex != i, yiledChildIndex == i, 3);
+    }
+
+    ProcedureMetrics proc = new ProcedureMetrics(true, true, 3, subprocs);
+    long id = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertNotEquals("ProcId zero!", 0, id);
+    beginCount += subProcCount + 1;
+    successCount += subProcCount - (failChildIndex + 1);
+    if (failChildIndex >= 0) {
+      failedCount += subProcCount + 1;
+    } else {
+      successCount++;
+    }
+    ProcedureTestingUtility.waitProcedure(procExecutor, proc);
+    assertEquals("beginCount doesn't match!", beginCount, proc.beginCount);
+    assertEquals("successCount doesn't match!", successCount, proc.successCount);
+    assertEquals("failedCont doesn't match!", failedCount, proc.failedCount);
+  }
+
+  private static class TestProcEnv {
+    public boolean toggleKillBeforeStoreUpdate = false;
+    public boolean triggerRollbackOnChild = false;
+  }
+
+  public static class ProcedureMetrics extends SequentialProcedure<TestProcEnv> {
+    public static long beginCount = 0;
+    public static long successCount = 0;
+    public static long failedCount = 0;
+
+    private boolean success;
+    private boolean yield;
+    private int yieldCount;
+    private int yieldNum;
+
+    private ProcedureMetrics[] subprocs = null;
+
+    public ProcedureMetrics() {
+      this(true);
+    }
+
+    public ProcedureMetrics(boolean success) {
+      this(success, true);
+    }
+
+    public ProcedureMetrics(boolean success, boolean yield) {
+      this(success, yield, 1);
+    }
+
+    public ProcedureMetrics(boolean success, boolean yield, int yieldCount) {
+      this(success, yield, yieldCount, null);
+    }
+
+    public ProcedureMetrics(boolean success, ProcedureMetrics[] subprocs) {
+      this(success, false, 1, subprocs);
+    }
+
+    public ProcedureMetrics(boolean success, boolean yield, int yieldCount,
+                            ProcedureMetrics[] subprocs) {
+      this.success = success;
+      this.yield = yield;
+      this.yieldCount = yieldCount;
+      this.subprocs = subprocs;
+      yieldNum = 0;
+    }
+
+    @Override
+    protected void updateMetricsOnSubmit(TestProcEnv env) {
+      beginCount++;
+    }
+
+    @Override
+    protected Procedure[] execute(TestProcEnv env) throws ProcedureYieldException,
+        ProcedureSuspendedException, InterruptedException {
+      if (this.yield) {
+        if (yieldNum < yieldCount) {
+          yieldNum++;
+          throw new ProcedureYieldException();
+        }
+      }
+      if (!this.success) {
+        setFailure("Failed", new InterruptedException("Failed"));
+        return null;
+      }
+      return subprocs;
+    }
+
+    @Override
+    protected void rollback(TestProcEnv env) throws IOException, InterruptedException {
+
+    }
+
+    @Override
+    protected boolean abort(TestProcEnv env) {
+      return false;
+    }
+
+    @Override
+    protected void updateMetricsOnFinish(final TestProcEnv env, final long time,
+    boolean success) {
+      if (success) {
+        successCount++;
+      } else {
+        failedCount++;
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
index 4b36c76..82b767e 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.procedure2;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 0071bef..8ff19b2 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -58948,13 +58948,13 @@ public final class MasterProtos {
     org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.State getState();
 
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    boolean hasStartTime();
+    boolean hasSubmittedTime();
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    long getStartTime();
+    long getSubmittedTime();
 
     /**
      * <code>optional uint64 last_update = 3;</code>
@@ -59000,7 +59000,7 @@ public final class MasterProtos {
     }
     private GetProcedureResultResponse() {
       state_ = 0;
-      startTime_ = 0L;
+      submittedTime_ = 0L;
       lastUpdate_ = 0L;
       result_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
@@ -59046,7 +59046,7 @@ public final class MasterProtos {
             }
             case 16: {
               bitField0_ |= 0x00000002;
-              startTime_ = input.readUInt64();
+              submittedTime_ = input.readUInt64();
               break;
             }
             case 24: {
@@ -59212,19 +59212,19 @@ public final class MasterProtos {
       return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.State.NOT_FOUND : result;
     }
 
-    public static final int START_TIME_FIELD_NUMBER = 2;
-    private long startTime_;
+    public static final int SUBMITTED_TIME_FIELD_NUMBER = 2;
+    private long submittedTime_;
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    public boolean hasStartTime() {
+    public boolean hasSubmittedTime() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional uint64 start_time = 2;</code>
+     * <code>optional uint64 submitted_time = 2;</code>
      */
-    public long getStartTime() {
-      return startTime_;
+    public long getSubmittedTime() {
+      return submittedTime_;
     }
 
     public static final int LAST_UPDATE_FIELD_NUMBER = 3;
@@ -59298,7 +59298,7 @@ public final class MasterProtos {
         output.writeEnum(1, state_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt64(2, startTime_);
+        output.writeUInt64(2, submittedTime_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeUInt64(3, lastUpdate_);
@@ -59323,7 +59323,7 @@ public final class MasterProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(2, startTime_);
+          .computeUInt64Size(2, submittedTime_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
@@ -59358,10 +59358,10 @@ public final class MasterProtos {
       if (hasState()) {
         result = result && state_ == other.state_;
       }
-      result = result && (hasStartTime() == other.hasStartTime());
-      if (hasStartTime()) {
-        result = result && (getStartTime()
-            == other.getStartTime());
+      result = result && (hasSubmittedTime() == other.hasSubmittedTime());
+      if (hasSubmittedTime()) {
+        result = result && (getSubmittedTime()
+            == other.getSubmittedTime());
       }
       result = result && (hasLastUpdate() == other.hasLastUpdate());
       if (hasLastUpdate()) {
@@ -59393,10 +59393,10 @@ public final class MasterProtos {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + state_;
       }
-      if (hasStartTime()) {
-        hash = (37 * hash) + START_TIME_FIELD_NUMBER;
+      if (hasSubmittedTime()) {
+        hash = (37 * hash) + SUBMITTED_TIME_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getStartTime());
+            getSubmittedTime());
       }
       if (hasLastUpdate()) {
         hash = (37 * hash) + LAST_UPDATE_FIELD_NUMBER;
@@ -59532,7 +59532,7 @@ public final class MasterProtos {
         super.clear();
         state_ = 0;
         bitField0_ = (bitField0_ & ~0x00000001);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
         lastUpdate_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -59575,7 +59575,7 @@ public final class MasterProtos {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.startTime_ = startTime_;
+        result.submittedTime_ = submittedTime_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
           to_bitField0_ |= 0x00000004;
         }
@@ -59637,8 +59637,8 @@ public final class MasterProtos {
         if (other.hasState()) {
           setState(other.getState());
         }
-        if (other.hasStartTime()) {
-          setStartTime(other.getStartTime());
+        if (other.hasSubmittedTime()) {
+          setSubmittedTime(other.getSubmittedTime());
         }
         if (other.hasLastUpdate()) {
           setLastUpdate(other.getLastUpdate());
@@ -59716,34 +59716,34 @@ public final class MasterProtos {
         return this;
       }
 
-      private long startTime_ ;
+      private long submittedTime_ ;
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public boolean hasStartTime() {
+      public boolean hasSubmittedTime() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public long getStartTime() {
-        return startTime_;
+      public long getSubmittedTime() {
+        return submittedTime_;
       }
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public Builder setStartTime(long value) {
+      public Builder setSubmittedTime(long value) {
         bitField0_ |= 0x00000002;
-        startTime_ = value;
+        submittedTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional uint64 start_time = 2;</code>
+       * <code>optional uint64 submitted_time = 2;</code>
        */
-      public Builder clearStartTime() {
+      public Builder clearSubmittedTime() {
         bitField0_ = (bitField0_ & ~0x00000002);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         onChanged();
         return this;
       }
@@ -76629,214 +76629,214 @@ public final class MasterProtos {
       "Description\"`\n\027IsProcedureDoneResponse\022\023" +
       "\n\004done\030\001 \001(\010:\005false\0220\n\010snapshot\030\002 \001(\0132\036.",
       "hbase.pb.ProcedureDescription\",\n\031GetProc" +
-      "edureResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032" +
+      "edureResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\375\001\n\032" +
       "GetProcedureResultResponse\0229\n\005state\030\001 \002(" +
       "\0162*.hbase.pb.GetProcedureResultResponse." +
-      "State\022\022\n\nstart_time\030\002 \001(\004\022\023\n\013last_update" +
-      "\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224\n\texception\030\005 \001(" +
-      "\0132!.hbase.pb.ForeignExceptionMessage\"1\n\005" +
-      "State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FIN" +
-      "ISHED\020\002\"M\n\025AbortProcedureRequest\022\017\n\007proc" +
-      "_id\030\001 \002(\004\022#\n\025mayInterruptIfRunning\030\002 \001(\010",
-      ":\004true\"6\n\026AbortProcedureResponse\022\034\n\024is_p" +
-      "rocedure_aborted\030\001 \002(\010\"\027\n\025ListProcedures" +
-      "Request\"@\n\026ListProceduresResponse\022&\n\tpro" +
-      "cedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001\n\017Se" +
-      "tQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser" +
-      "_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\ntable" +
-      "_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n\nremo" +
-      "ve_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010\022+\n\010" +
-      "throttle\030\007 \001(\0132\031.hbase.pb.ThrottleReques" +
-      "t\"\022\n\020SetQuotaResponse\"J\n\037MajorCompaction",
-      "TimestampRequest\022\'\n\ntable_name\030\001 \002(\0132\023.h" +
-      "base.pb.TableName\"U\n(MajorCompactionTime" +
-      "stampForRegionRequest\022)\n\006region\030\001 \002(\0132\031." +
-      "hbase.pb.RegionSpecifier\"@\n MajorCompact" +
-      "ionTimestampResponse\022\034\n\024compaction_times" +
-      "tamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesReques" +
-      "t\"\354\001\n\034SecurityCapabilitiesResponse\022G\n\014ca" +
-      "pabilities\030\001 \003(\01621.hbase.pb.SecurityCapa" +
-      "bilitiesResponse.Capability\"\202\001\n\nCapabili" +
-      "ty\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SECURE_",
-      "AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022C" +
-      "ELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBILITY\020\004" +
-      "\"\"\n ListDrainingRegionServersRequest\"N\n!" +
-      "ListDrainingRegionServersResponse\022)\n\013ser" +
-      "ver_name\030\001 \003(\0132\024.hbase.pb.ServerName\"F\n\031" +
-      "DrainRegionServersRequest\022)\n\013server_name" +
-      "\030\001 \003(\0132\024.hbase.pb.ServerName\"\034\n\032DrainReg" +
-      "ionServersResponse\"P\n#RemoveDrainFromReg" +
-      "ionServersRequest\022)\n\013server_name\030\001 \003(\0132\024" +
-      ".hbase.pb.ServerName\"&\n$RemoveDrainFromR",
-      "egionServersResponse*(\n\020MasterSwitchType" +
-      "\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n\rMasterService" +
-      "\022e\n\024GetSchemaAlterStatus\022%.hbase.pb.GetS" +
-      "chemaAlterStatusRequest\032&.hbase.pb.GetSc" +
-      "hemaAlterStatusResponse\022b\n\023GetTableDescr" +
-      "iptors\022$.hbase.pb.GetTableDescriptorsReq" +
-      "uest\032%.hbase.pb.GetTableDescriptorsRespo" +
-      "nse\022P\n\rGetTableNames\022\036.hbase.pb.GetTable" +
-      "NamesRequest\032\037.hbase.pb.GetTableNamesRes" +
-      "ponse\022Y\n\020GetClusterStatus\022!.hbase.pb.Get",
-      "ClusterStatusRequest\032\".hbase.pb.GetClust" +
-      "erStatusResponse\022V\n\017IsMasterRunning\022 .hb" +
-      "ase.pb.IsMasterRunningRequest\032!.hbase.pb" +
-      ".IsMasterRunningResponse\022D\n\tAddColumn\022\032." +
-      "hbase.pb.AddColumnRequest\032\033.hbase.pb.Add" +
-      "ColumnResponse\022M\n\014DeleteColumn\022\035.hbase.p" +
-      "b.DeleteColumnRequest\032\036.hbase.pb.DeleteC" +
-      "olumnResponse\022M\n\014ModifyColumn\022\035.hbase.pb" +
-      ".ModifyColumnRequest\032\036.hbase.pb.ModifyCo" +
-      "lumnResponse\022G\n\nMoveRegion\022\033.hbase.pb.Mo",
-      "veRegionRequest\032\034.hbase.pb.MoveRegionRes" +
-      "ponse\022\\\n\021MergeTableRegions\022\".hbase.pb.Me" +
-      "rgeTableRegionsRequest\032#.hbase.pb.MergeT" +
-      "ableRegionsResponse\022M\n\014AssignRegion\022\035.hb" +
-      "ase.pb.AssignRegionRequest\032\036.hbase.pb.As" +
-      "signRegionResponse\022S\n\016UnassignRegion\022\037.h" +
-      "base.pb.UnassignRegionRequest\032 .hbase.pb" +
-      ".UnassignRegionResponse\022P\n\rOfflineRegion" +
-      "\022\036.hbase.pb.OfflineRegionRequest\032\037.hbase" +
-      ".pb.OfflineRegionResponse\022J\n\013DeleteTable",
-      "\022\034.hbase.pb.DeleteTableRequest\032\035.hbase.p" +
-      "b.DeleteTableResponse\022P\n\rtruncateTable\022\036" +
-      ".hbase.pb.TruncateTableRequest\032\037.hbase.p" +
-      "b.TruncateTableResponse\022J\n\013EnableTable\022\034" +
-      ".hbase.pb.EnableTableRequest\032\035.hbase.pb." +
-      "EnableTableResponse\022M\n\014DisableTable\022\035.hb" +
-      "ase.pb.DisableTableRequest\032\036.hbase.pb.Di" +
-      "sableTableResponse\022J\n\013ModifyTable\022\034.hbas" +
-      "e.pb.ModifyTableRequest\032\035.hbase.pb.Modif" +
-      "yTableResponse\022J\n\013CreateTable\022\034.hbase.pb",
-      ".CreateTableRequest\032\035.hbase.pb.CreateTab" +
-      "leResponse\022A\n\010Shutdown\022\031.hbase.pb.Shutdo" +
-      "wnRequest\032\032.hbase.pb.ShutdownResponse\022G\n" +
-      "\nStopMaster\022\033.hbase.pb.StopMasterRequest" +
-      "\032\034.hbase.pb.StopMasterResponse\022h\n\031IsMast" +
-      "erInMaintenanceMode\022$.hbase.pb.IsInMaint" +
-      "enanceModeRequest\032%.hbase.pb.IsInMainten" +
-      "anceModeResponse\022>\n\007Balance\022\030.hbase.pb.B" +
-      "alanceRequest\032\031.hbase.pb.BalanceResponse" +
-      "\022_\n\022SetBalancerRunning\022#.hbase.pb.SetBal",
-      "ancerRunningRequest\032$.hbase.pb.SetBalanc" +
-      "erRunningResponse\022\\\n\021IsBalancerEnabled\022\"" +
-      ".hbase.pb.IsBalancerEnabledRequest\032#.hba" +
-      "se.pb.IsBalancerEnabledResponse\022k\n\026SetSp" +
-      "litOrMergeEnabled\022\'.hbase.pb.SetSplitOrM" +
-      "ergeEnabledRequest\032(.hbase.pb.SetSplitOr" +
-      "MergeEnabledResponse\022h\n\025IsSplitOrMergeEn" +
-      "abled\022&.hbase.pb.IsSplitOrMergeEnabledRe" +
-      "quest\032\'.hbase.pb.IsSplitOrMergeEnabledRe" +
-      "sponse\022D\n\tNormalize\022\032.hbase.pb.Normalize",
-      "Request\032\033.hbase.pb.NormalizeResponse\022e\n\024" +
-      "SetNormalizerRunning\022%.hbase.pb.SetNorma" +
-      "lizerRunningRequest\032&.hbase.pb.SetNormal" +
-      "izerRunningResponse\022b\n\023IsNormalizerEnabl" +
-      "ed\022$.hbase.pb.IsNormalizerEnabledRequest" +
-      "\032%.hbase.pb.IsNormalizerEnabledResponse\022" +
-      "S\n\016RunCatalogScan\022\037.hbase.pb.RunCatalogS" +
-      "canRequest\032 .hbase.pb.RunCatalogScanResp" +
-      "onse\022e\n\024EnableCatalogJanitor\022%.hbase.pb." +
-      "EnableCatalogJanitorRequest\032&.hbase.pb.E",
-      "nableCatalogJanitorResponse\022n\n\027IsCatalog" +
-      "JanitorEnabled\022(.hbase.pb.IsCatalogJanit" +
-      "orEnabledRequest\032).hbase.pb.IsCatalogJan" +
-      "itorEnabledResponse\022V\n\017RunCleanerChore\022 " +
-      ".hbase.pb.RunCleanerChoreRequest\032!.hbase" +
-      ".pb.RunCleanerChoreResponse\022k\n\026SetCleane" +
-      "rChoreRunning\022\'.hbase.pb.SetCleanerChore" +
-      "RunningRequest\032(.hbase.pb.SetCleanerChor" +
-      "eRunningResponse\022h\n\025IsCleanerChoreEnable" +
-      "d\022&.hbase.pb.IsCleanerChoreEnabledReques",
-      "t\032\'.hbase.pb.IsCleanerChoreEnabledRespon" +
-      "se\022^\n\021ExecMasterService\022#.hbase.pb.Copro" +
-      "cessorServiceRequest\032$.hbase.pb.Coproces" +
-      "sorServiceResponse\022A\n\010Snapshot\022\031.hbase.p" +
-      "b.SnapshotRequest\032\032.hbase.pb.SnapshotRes" +
-      "ponse\022h\n\025GetCompletedSnapshots\022&.hbase.p" +
-      "b.GetCompletedSnapshotsRequest\032\'.hbase.p" +
-      "b.GetCompletedSnapshotsResponse\022S\n\016Delet" +
-      "eSnapshot\022\037.hbase.pb.DeleteSnapshotReque" +
-      "st\032 .hbase.pb.DeleteSnapshotResponse\022S\n\016",
-      "IsSnapshotDone\022\037.hbase.pb.IsSnapshotDone" +
-      "Request\032 .hbase.pb.IsSnapshotDoneRespons" +
-      "e\022V\n\017RestoreSnapshot\022 .hbase.pb.RestoreS" +
-      "napshotRequest\032!.hbase.pb.RestoreSnapsho" +
-      "tResponse\022P\n\rExecProcedure\022\036.hbase.pb.Ex" +
-      "ecProcedureRequest\032\037.hbase.pb.ExecProced" +
-      "ureResponse\022W\n\024ExecProcedureWithRet\022\036.hb" +
-      "ase.pb.ExecProcedureRequest\032\037.hbase.pb.E" +
-      "xecProcedureResponse\022V\n\017IsProcedureDone\022" +
-      " .hbase.pb.IsProcedureDoneRequest\032!.hbas",
-      "e.pb.IsProcedureDoneResponse\022V\n\017ModifyNa" +
-      "mespace\022 .hbase.pb.ModifyNamespaceReques" +
-      "t\032!.hbase.pb.ModifyNamespaceResponse\022V\n\017" +
-      "CreateNamespace\022 .hbase.pb.CreateNamespa" +
-      "ceRequest\032!.hbase.pb.CreateNamespaceResp" +
-      "onse\022V\n\017DeleteNamespace\022 .hbase.pb.Delet" +
-      "eNamespaceRequest\032!.hbase.pb.DeleteNames" +
-      "paceResponse\022k\n\026GetNamespaceDescriptor\022\'" +
-      ".hbase.pb.GetNamespaceDescriptorRequest\032" +
-      "(.hbase.pb.GetNamespaceDescriptorRespons",
-      "e\022q\n\030ListNamespaceDescriptors\022).hbase.pb" +
-      ".ListNamespaceDescriptorsRequest\032*.hbase" +
-      ".pb.ListNamespaceDescriptorsResponse\022\206\001\n" +
-      "\037ListTableDescriptorsByNamespace\0220.hbase" +
-      ".pb.ListTableDescriptorsByNamespaceReque" +
-      "st\0321.hbase.pb.ListTableDescriptorsByName" +
-      "spaceResponse\022t\n\031ListTableNamesByNamespa" +
-      "ce\022*.hbase.pb.ListTableNamesByNamespaceR" +
-      "equest\032+.hbase.pb.ListTableNamesByNamesp" +
-      "aceResponse\022P\n\rGetTableState\022\036.hbase.pb.",
-      "GetTableStateRequest\032\037.hbase.pb.GetTable" +
-      "StateResponse\022A\n\010SetQuota\022\031.hbase.pb.Set" +
-      "QuotaRequest\032\032.hbase.pb.SetQuotaResponse" +
-      "\022x\n\037getLastMajorCompactionTimestamp\022).hb" +
-      "ase.pb.MajorCompactionTimestampRequest\032*" +
-      ".hbase.pb.MajorCompactionTimestampRespon" +
-      "se\022\212\001\n(getLastMajorCompactionTimestampFo" +
-      "rRegion\0222.hbase.pb.MajorCompactionTimest" +
-      "ampForRegionRequest\032*.hbase.pb.MajorComp" +
-      "actionTimestampResponse\022_\n\022getProcedureR",
-      "esult\022#.hbase.pb.GetProcedureResultReque" +
-      "st\032$.hbase.pb.GetProcedureResultResponse" +
-      "\022h\n\027getSecurityCapabilities\022%.hbase.pb.S" +
-      "ecurityCapabilitiesRequest\032&.hbase.pb.Se" +
-      "curityCapabilitiesResponse\022S\n\016AbortProce" +
-      "dure\022\037.hbase.pb.AbortProcedureRequest\032 ." +
-      "hbase.pb.AbortProcedureResponse\022S\n\016ListP" +
-      "rocedures\022\037.hbase.pb.ListProceduresReque" +
-      "st\032 .hbase.pb.ListProceduresResponse\022_\n\022" +
-      "AddReplicationPeer\022#.hbase.pb.AddReplica",
-      "tionPeerRequest\032$.hbase.pb.AddReplicatio" +
-      "nPeerResponse\022h\n\025RemoveReplicationPeer\022&" +
-      ".hbase.pb.RemoveReplicationPeerRequest\032\'" +
-      ".hbase.pb.RemoveReplicationPeerResponse\022" +
-      "h\n\025EnableReplicationPeer\022&.hbase.pb.Enab" +
-      "leReplicationPeerRequest\032\'.hbase.pb.Enab" +
-      "leReplicationPeerResponse\022k\n\026DisableRepl" +
-      "icationPeer\022\'.hbase.pb.DisableReplicatio" +
-      "nPeerRequest\032(.hbase.pb.DisableReplicati" +
-      "onPeerResponse\022q\n\030GetReplicationPeerConf",
-      "ig\022).hbase.pb.GetReplicationPeerConfigRe" +
-      "quest\032*.hbase.pb.GetReplicationPeerConfi" +
-      "gResponse\022z\n\033UpdateReplicationPeerConfig" +
-      "\022,.hbase.pb.UpdateReplicationPeerConfigR" +
-      "equest\032-.hbase.pb.UpdateReplicationPeerC" +
-      "onfigResponse\022e\n\024ListReplicationPeers\022%." +
-      "hbase.pb.ListReplicationPeersRequest\032&.h" +
-      "base.pb.ListReplicationPeersResponse\022t\n\031" +
-      "listDrainingRegionServers\022*.hbase.pb.Lis" +
-      "tDrainingRegionServersRequest\032+.hbase.pb",
-      ".ListDrainingRegionServersResponse\022_\n\022dr" +
-      "ainRegionServers\022#.hbase.pb.DrainRegionS" +
-      "erversRequest\032$.hbase.pb.DrainRegionServ" +
-      "ersResponse\022}\n\034removeDrainFromRegionServ" +
-      "ers\022-.hbase.pb.RemoveDrainFromRegionServ" +
-      "ersRequest\032..hbase.pb.RemoveDrainFromReg" +
-      "ionServersResponseBI\n1org.apache.hadoop." +
-      "hbase.shaded.protobuf.generatedB\014MasterP" +
-      "rotosH\001\210\001\001\240\001\001"
+      "State\022\026\n\016submitted_time\030\002 \001(\004\022\023\n\013last_up" +
+      "date\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224\n\texception\030" +
+      "\005 \001(\0132!.hbase.pb.ForeignExceptionMessage" +
+      "\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n" +
+      "\010FINISHED\020\002\"M\n\025AbortProcedureRequest\022\017\n\007" +
+      "proc_id\030\001 \002(\004\022#\n\025mayInterruptIfRunning\030\002",
+      " \001(\010:\004true\"6\n\026AbortProcedureResponse\022\034\n\024" +
+      "is_procedure_aborted\030\001 \002(\010\"\027\n\025ListProced" +
+      "uresRequest\"@\n\026ListProceduresResponse\022&\n" +
+      "\tprocedure\030\001 \003(\0132\023.hbase.pb.Procedure\"\315\001" +
+      "\n\017SetQuotaRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\n" +
+      "user_group\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\nt" +
+      "able_name\030\004 \001(\0132\023.hbase.pb.TableName\022\022\n\n" +
+      "remove_all\030\005 \001(\010\022\026\n\016bypass_globals\030\006 \001(\010" +
+      "\022+\n\010throttle\030\007 \001(\0132\031.hbase.pb.ThrottleRe" +
+      "quest\"\022\n\020SetQuotaResponse\"J\n\037MajorCompac",
+      "tionTimestampRequest\022\'\n\ntable_name\030\001 \002(\013" +
+      "2\023.hbase.pb.TableName\"U\n(MajorCompaction" +
+      "TimestampForRegionRequest\022)\n\006region\030\001 \002(" +
+      "\0132\031.hbase.pb.RegionSpecifier\"@\n MajorCom" +
+      "pactionTimestampResponse\022\034\n\024compaction_t" +
+      "imestamp\030\001 \002(\003\"\035\n\033SecurityCapabilitiesRe" +
+      "quest\"\354\001\n\034SecurityCapabilitiesResponse\022G" +
+      "\n\014capabilities\030\001 \003(\01621.hbase.pb.Security" +
+      "CapabilitiesResponse.Capability\"\202\001\n\nCapa" +
+      "bility\022\031\n\025SIMPLE_AUTHENTICATION\020\000\022\031\n\025SEC",
+      "URE_AUTHENTICATION\020\001\022\021\n\rAUTHORIZATION\020\002\022" +
+      "\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017CELL_VISIBILI" +
+      "TY\020\004\"\"\n ListDrainingRegionServersRequest" +
+      "\"N\n!ListDrainingRegionServersResponse\022)\n" +
+      "\013server_name\030\001 \003(\0132\024.hbase.pb.ServerName" +
+      "\"F\n\031DrainRegionServersRequest\022)\n\013server_" +
+      "name\030\001 \003(\0132\024.hbase.pb.ServerName\"\034\n\032Drai" +
+      "nRegionServersResponse\"P\n#RemoveDrainFro" +
+      "mRegionServersRequest\022)\n\013server_name\030\001 \003" +
+      "(\0132\024.hbase.pb.ServerName\"&\n$RemoveDrainF",
+      "romRegionServersResponse*(\n\020MasterSwitch" +
+      "Type\022\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\3013\n\rMasterSer" +
+      "vice\022e\n\024GetSchemaAlterStatus\022%.hbase.pb." +
+      "GetSchemaAlterStatusRequest\032&.hbase.pb.G" +
+      "etSchemaAlterStatusResponse\022b\n\023GetTableD" +
+      "escriptors\022$.hbase.pb.GetTableDescriptor" +
+      "sRequest\032%.hbase.pb.GetTableDescriptorsR" +
+      "esponse\022P\n\rGetTableNames\022\036.hbase.pb.GetT" +
+      "ableNamesRequest\032\037.hbase.pb.GetTableName" +
+      "sResponse\022Y\n\020GetClusterStatus\022!.hbase.pb",
+      ".GetClusterStatusRequest\032\".hbase.pb.GetC" +
+      "lusterStatusResponse\022V\n\017IsMasterRunning\022" +
+      " .hbase.pb.IsMasterRunningRequest\032!.hbas" +
+      "e.pb.IsMasterRunningResponse\022D\n\tAddColum" +
+      "n\022\032.hbase.pb.AddColumnRequest\032\033.hbase.pb" +
+      ".AddColumnResponse\022M\n\014DeleteColumn\022\035.hba" +
+      "se.pb.DeleteColumnRequest\032\036.hbase.pb.Del" +
+      "eteColumnResponse\022M\n\014ModifyColumn\022\035.hbas" +
+      "e.pb.ModifyColumnRequest\032\036.hbase.pb.Modi" +
+      "fyColumnResponse\022G\n\nMoveRegion\022\033.hbase.p",
+      "b.MoveRegionRequest\032\034.hbase.pb.MoveRegio" +
+      "nResponse\022\\\n\021MergeTableRegions\022\".hbase.p" +
+      "b.MergeTableRegionsRequest\032#.hbase.pb.Me" +
+      "rgeTableRegionsResponse\022M\n\014AssignRegion\022" +
+      "\035.hbase.pb.AssignRegionRequest\032\036.hbase.p" +
+      "b.AssignRegionResponse\022S\n\016UnassignRegion" +
+      "\022\037.hbase.pb.UnassignRegionRequest\032 .hbas" +
+      "e.pb.UnassignRegionResponse\022P\n\rOfflineRe" +
+      "gion\022\036.hbase.pb.OfflineRegionRequest\032\037.h" +
+      "base.pb.OfflineRegionResponse\022J\n\013DeleteT",
+      "able\022\034.hbase.pb.DeleteTableRequest\032\035.hba" +
+      "se.pb.DeleteTableResponse\022P\n\rtruncateTab" +
+      "le\022\036.hbase.pb.TruncateTableRequest\032\037.hba" +
+      "se.pb.TruncateTableResponse\022J\n\013EnableTab" +
+      "le\022\034.hbase.pb.EnableTableRequest\032\035.hbase" +
+      ".pb.EnableTableResponse\022M\n\014DisableTable\022" +
+      "\035.hbase.pb.DisableTableRequest\032\036.hbase.p" +
+      "b.DisableTableResponse\022J\n\013ModifyTable\022\034." +
+      "hbase.pb.ModifyTableRequest\032\035.hbase.pb.M" +
+      "odifyTableResponse\022J\n\013CreateTable\022\034.hbas",
+      "e.pb.CreateTableRequest\032\035.hbase.pb.Creat" +
+      "eTableResponse\022A\n\010Shutdown\022\031.hbase.pb.Sh" +
+      "utdownRequest\032\032.hbase.pb.ShutdownRespons" +
+      "e\022G\n\nStopMaster\022\033.hbase.pb.StopMasterReq" +
+      "uest\032\034.hbase.pb.StopMasterResponse\022h\n\031Is" +
+      "MasterInMaintenanceMode\022$.hbase.pb.IsInM" +
+      "aintenanceModeRequest\032%.hbase.pb.IsInMai" +
+      "ntenanceModeResponse\022>\n\007Balance\022\030.hbase." +
+      "pb.BalanceRequest\032\031.hbase.pb.BalanceResp" +
+      "onse\022_\n\022SetBalancerRunning\022#.hbase.pb.Se",
+      "tBalancerRunningRequest\032$.hbase.pb.SetBa" +
+      "lancerRunningResponse\022\\\n\021IsBalancerEnabl" +
+      "ed\022\".hbase.pb.IsBalancerEnabledRequest\032#" +
+      ".hbase.pb.IsBalancerEnabledResponse\022k\n\026S" +
+      "etSplitOrMergeEnabled\022\'.hbase.pb.SetSpli" +
+      "tOrMergeEnabledRequest\032(.hbase.pb.SetSpl" +
+      "itOrMergeEnabledResponse\022h\n\025IsSplitOrMer" +
+      "geEnabled\022&.hbase.pb.IsSplitOrMergeEnabl" +
+      "edRequest\032\'.hbase.pb.IsSplitOrMergeEnabl" +
+      "edResponse\022D\n\tNormalize\022\032.hbase.pb.Norma",
+      "lizeRequest\032\033.hbase.pb.NormalizeResponse" +
+      "\022e\n\024SetNormalizerRunning\022%.hbase.pb.SetN" +
+      "ormalizerRunningRequest\032&.hbase.pb.SetNo" +
+      "rmalizerRunningResponse\022b\n\023IsNormalizerE" +
+      "nabled\022$.hbase.pb.IsNormalizerEnabledReq" +
+      "uest\032%.hbase.pb.IsNormalizerEnabledRespo" +
+      "nse\022S\n\016RunCatalogScan\022\037.hbase.pb.RunCata" +
+      "logScanRequest\032 .hbase.pb.RunCatalogScan" +
+      "Response\022e\n\024EnableCatalogJanitor\022%.hbase" +
+      ".pb.EnableCatalogJanitorRequest\032&.hbase.",
+      "pb.EnableCatalogJanitorResponse\022n\n\027IsCat" +
+      "alogJanitorEnabled\022(.hbase.pb.IsCatalogJ" +
+      "anitorEnabledRequest\032).hbase.pb.IsCatalo" +
+      "gJanitorEnabledResponse\022V\n\017RunCleanerCho" +
+      "re\022 .hbase.pb.RunCleanerChoreRequest\032!.h" +
+      "base.pb.RunCleanerChoreResponse\022k\n\026SetCl" +
+      "eanerChoreRunning\022\'.hbase.pb.SetCleanerC" +
+      "horeRunningRequest\032(.hbase.pb.SetCleaner" +
+      "ChoreRunningResponse\022h\n\025IsCleanerChoreEn" +
+      "abled\022&.hbase.pb.IsCleanerChoreEnabledRe",
+      "quest\032\'.hbase.pb.IsCleanerChoreEnabledRe" +
+      "sponse\022^\n\021ExecMasterService\022#.hbase.pb.C" +
+      "oprocessorServiceRequest\032$.hbase.pb.Copr" +
+      "ocessorServiceResponse\022A\n\010Snapshot\022\031.hba" +
+      "se.pb.SnapshotRequest\032\032.hbase.pb.Snapsho" +
+      "tResponse\022h\n\025GetCompletedSnapshots\022&.hba" +
+      "se.pb.GetCompletedSnapshotsRequest\032\'.hba" +
+      "se.pb.GetCompletedSnapshotsResponse\022S\n\016D" +
+      "eleteSnapshot\022\037.hbase.pb.DeleteSnapshotR" +
+      "equest\032 .hbase.pb.DeleteSnapshotResponse",
+      "\022S\n\016IsSnapshotDone\022\037.hbase.pb.IsSnapshot" +
+      "DoneRequest\032 .hbase.pb.IsSnapshotDoneRes" +
+      "ponse\022V\n\017RestoreSnapshot\022 .hbase.pb.Rest" +
+      "oreSnapshotRequest\032!.hbase.pb.RestoreSna" +
+      "pshotResponse\022P\n\rExecProcedure\022\036.hbase.p" +
+      "b.ExecProcedureRequest\032\037.hbase.pb.ExecPr" +
+      "ocedureResponse\022W\n\024ExecProcedureWithRet\022" +
+      "\036.hbase.pb.ExecProcedureRequest\032\037.hbase." +
+      "pb.ExecProcedureResponse\022V\n\017IsProcedureD" +
+      "one\022 .hbase.pb.IsProcedureDoneRequest\032!.",
+      "hbase.pb.IsProcedureDoneResponse\022V\n\017Modi" +
+      "fyNamespace\022 .hbase.pb.ModifyNamespaceRe" +
+      "quest\032!.hbase.pb.ModifyNamespaceResponse" +
+      "\022V\n\017CreateNamespace\022 .hbase.pb.CreateNam" +
+      "espaceRequest\032!.hbase.pb.CreateNamespace" +
+      "Response\022V\n\017DeleteNamespace\022 .hbase.pb.D" +
+      "eleteNamespaceRequest\032!.hbase.pb.DeleteN" +
+      "amespaceResponse\022k\n\026GetNamespaceDescript" +
+      "or\022\'.hbase.pb.GetNamespaceDescriptorRequ" +
+      "est\032(.hbase.pb.GetNamespaceDescriptorRes",
+      "ponse\022q\n\030ListNamespaceDescriptors\022).hbas" +
+      "e.pb.ListNamespaceDescriptorsRequest\032*.h" +
+      "base.pb.ListNamespaceDescriptorsResponse" +
+      "\022\206\001\n\037ListTableDescriptorsByNamespace\0220.h" +
+      "base.pb.ListTableDescriptorsByNamespaceR" +
+      "equest\0321.hbase.pb.ListTableDescriptorsBy" +
+      "NamespaceResponse\022t\n\031ListTableNamesByNam" +
+      "espace\022*.hbase.pb.ListTableNamesByNamesp" +
+      "aceRequest\032+.hbase.pb.ListTableNamesByNa" +
+      "mespaceResponse\022P\n\rGetTableState\022\036.hbase",
+      ".pb.GetTableStateRequest\032\037.hbase.pb.GetT" +
+      "ableStateResponse\022A\n\010SetQuota\022\031.hbase.pb" +
+      ".SetQuotaRequest\032\032.hbase.pb.SetQuotaResp" +
+      "onse\022x\n\037getLastMajorCompactionTimestamp\022" +
+      ").hbase.pb.MajorCompactionTimestampReque" +
+      "st\032*.hbase.pb.MajorCompactionTimestampRe" +
+      "sponse\022\212\001\n(getLastMajorCompactionTimesta" +
+      "mpForRegion\0222.hbase.pb.MajorCompactionTi" +
+      "mestampForRegionRequest\032*.hbase.pb.Major" +
+      "CompactionTimestampResponse\022_\n\022getProced",
+      "ureResult\022#.hbase.pb.GetProcedureResultR" +
+      "equest\032$.hbase.pb.GetProcedureResultResp" +
+      "onse\022h\n\027getSecurityCapabilities\022%.hbase." +
+      "pb.SecurityCapabilitiesRequest\032&.hbase.p" +
+      "b.SecurityCapabilitiesResponse\022S\n\016AbortP" +
+      "rocedure\022\037.hbase.pb.AbortProcedureReques" +
+      "t\032 .hbase.pb.AbortProcedureResponse\022S\n\016L" +
+      "istProcedures\022\037.hbase.pb.ListProceduresR" +
+      "equest\032 .hbase.pb.ListProceduresResponse" +
+      "\022_\n\022AddReplicationPeer\022#.hbase.pb.AddRep",
+      "licationPeerRequest\032$.hbase.pb.AddReplic" +
+      "ationPeerResponse\022h\n\025RemoveReplicationPe" +
+      "er\022&.hbase.pb.RemoveReplicationPeerReque" +
+      "st\032\'.hbase.pb.RemoveReplicationPeerRespo" +
+      "nse\022h\n\025EnableReplicationPeer\022&.hbase.pb." +
+      "EnableReplicationPeerRequest\032\'.hbase.pb." +
+      "EnableReplicationPeerResponse\022k\n\026Disable" +
+      "ReplicationPeer\022\'.hbase.pb.DisableReplic" +
+      "ationPeerRequest\032(.hbase.pb.DisableRepli" +
+      "cationPeerResponse\022q\n\030GetReplicationPeer",
+      "Config\022).hbase.pb.GetReplicationPeerConf" +
+      "igRequest\032*.hbase.pb.GetReplicationPeerC" +
+      "onfigResponse\022z\n\033UpdateReplicationPeerCo" +
+      "nfig\022,.hbase.pb.UpdateReplicationPeerCon" +
+      "figRequest\032-.hbase.pb.UpdateReplicationP" +
+      "eerConfigResponse\022e\n\024ListReplicationPeer" +
+      "s\022%.hbase.pb.ListReplicationPeersRequest" +
+      "\032&.hbase.pb.ListReplicationPeersResponse" +
+      "\022t\n\031listDrainingRegionServers\022*.hbase.pb" +
+      ".ListDrainingRegionServersRequest\032+.hbas",
+      "e.pb.ListDrainingRegionServersResponse\022_" +
+      "\n\022drainRegionServers\022#.hbase.pb.DrainReg" +
+      "ionServersRequest\032$.hbase.pb.DrainRegion" +
+      "ServersResponse\022}\n\034removeDrainFromRegion" +
+      "Servers\022-.hbase.pb.RemoveDrainFromRegion" +
+      "ServersRequest\032..hbase.pb.RemoveDrainFro" +
+      "mRegionServersResponseBI\n1org.apache.had" +
+      "oop.hbase.shaded.protobuf.generatedB\014Mas" +
+      "terProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -77492,7 +77492,7 @@ public final class MasterProtos {
     internal_static_hbase_pb_GetProcedureResultResponse_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetProcedureResultResponse_descriptor,
-        new java.lang.String[] { "State", "StartTime", "LastUpdate", "Result", "Exception", });
+        new java.lang.String[] { "State", "SubmittedTime", "LastUpdate", "Result", "Exception", });
     internal_static_hbase_pb_AbortProcedureRequest_descriptor =
       getDescriptor().getMessageTypes().get(106);
     internal_static_hbase_pb_AbortProcedureRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
index 56d1e11..e6fe9b1 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
@@ -262,13 +262,13 @@ public final class ProcedureProtos {
     long getProcId();
 
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    boolean hasStartTime();
+    boolean hasSubmittedTime();
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    long getStartTime();
+    long getSubmittedTime();
 
     /**
      * <code>optional string owner = 5;</code>
@@ -449,7 +449,7 @@ public final class ProcedureProtos {
       className_ = "";
       parentId_ = 0L;
       procId_ = 0L;
-      startTime_ = 0L;
+      submittedTime_ = 0L;
       owner_ = "";
       state_ = 1;
       stackId_ = java.util.Collections.emptyList();
@@ -507,7 +507,7 @@ public final class ProcedureProtos {
             }
             case 32: {
               bitField0_ |= 0x00000008;
-              startTime_ = input.readUInt64();
+              submittedTime_ = input.readUInt64();
               break;
             }
             case 42: {
@@ -711,19 +711,19 @@ public final class ProcedureProtos {
       return procId_;
     }
 
-    public static final int START_TIME_FIELD_NUMBER = 4;
-    private long startTime_;
+    public static final int SUBMITTED_TIME_FIELD_NUMBER = 4;
+    private long submittedTime_;
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    public boolean hasStartTime() {
+    public boolean hasSubmittedTime() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>required uint64 start_time = 4;</code>
+     * <code>required uint64 submitted_time = 4;</code>
      */
-    public long getStartTime() {
-      return startTime_;
+    public long getSubmittedTime() {
+      return submittedTime_;
     }
 
     public static final int OWNER_FIELD_NUMBER = 5;
@@ -987,7 +987,7 @@ public final class ProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasStartTime()) {
+      if (!hasSubmittedTime()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -1015,7 +1015,7 @@ public final class ProcedureProtos {
         output.writeUInt64(3, procId_);
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt64(4, startTime_);
+        output.writeUInt64(4, submittedTime_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 5, owner_);
@@ -1068,7 +1068,7 @@ public final class ProcedureProtos {
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(4, startTime_);
+          .computeUInt64Size(4, submittedTime_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(5, owner_);
@@ -1146,10 +1146,10 @@ public final class ProcedureProtos {
         result = result && (getProcId()
             == other.getProcId());
       }
-      result = result && (hasStartTime() == other.hasStartTime());
-      if (hasStartTime()) {
-        result = result && (getStartTime()
-            == other.getStartTime());
+      result = result && (hasSubmittedTime() == other.hasSubmittedTime());
+      if (hasSubmittedTime()) {
+        result = result && (getSubmittedTime()
+            == other.getSubmittedTime());
       }
       result = result && (hasOwner() == other.hasOwner());
       if (hasOwner()) {
@@ -1222,10 +1222,10 @@ public final class ProcedureProtos {
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getProcId());
       }
-      if (hasStartTime()) {
-        hash = (37 * hash) + START_TIME_FIELD_NUMBER;
+      if (hasSubmittedTime()) {
+        hash = (37 * hash) + SUBMITTED_TIME_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getStartTime());
+            getSubmittedTime());
       }
       if (hasOwner()) {
         hash = (37 * hash) + OWNER_FIELD_NUMBER;
@@ -1400,7 +1400,7 @@ public final class ProcedureProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         procId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
         owner_ = "";
         bitField0_ = (bitField0_ & ~0x00000010);
@@ -1465,7 +1465,7 @@ public final class ProcedureProtos {
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000008;
         }
-        result.startTime_ = startTime_;
+        result.submittedTime_ = submittedTime_;
         if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
           to_bitField0_ |= 0x00000010;
         }
@@ -1564,8 +1564,8 @@ public final class ProcedureProtos {
         if (other.hasProcId()) {
           setProcId(other.getProcId());
         }
-        if (other.hasStartTime()) {
-          setStartTime(other.getStartTime());
+        if (other.hasSubmittedTime()) {
+          setSubmittedTime(other.getSubmittedTime());
         }
         if (other.hasOwner()) {
           bitField0_ |= 0x00000010;
@@ -1618,7 +1618,7 @@ public final class ProcedureProtos {
         if (!hasProcId()) {
           return false;
         }
-        if (!hasStartTime()) {
+        if (!hasSubmittedTime()) {
           return false;
         }
         if (!hasState()) {
@@ -1829,34 +1829,34 @@ public final class ProcedureProtos {
         return this;
       }
 
-      private long startTime_ ;
+      private long submittedTime_ ;
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public boolean hasStartTime() {
+      public boolean hasSubmittedTime() {
         return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public long getStartTime() {
-        return startTime_;
+      public long getSubmittedTime() {
+        return submittedTime_;
       }
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public Builder setStartTime(long value) {
+      public Builder setSubmittedTime(long value) {
         bitField0_ |= 0x00000008;
-        startTime_ = value;
+        submittedTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required uint64 start_time = 4;</code>
+       * <code>required uint64 submitted_time = 4;</code>
        */
-      public Builder clearStartTime() {
+      public Builder clearSubmittedTime() {
         bitField0_ = (bitField0_ & ~0x00000008);
-        startTime_ = 0L;
+        submittedTime_ = 0L;
         onChanged();
         return this;
       }
@@ -7743,38 +7743,38 @@ public final class ProcedureProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\017Procedure.proto\022\010hbase.pb\032\023ErrorHandli" +
-      "ng.proto\"\313\002\n\tProcedure\022\022\n\nclass_name\030\001 \002" +
-      "(\t\022\021\n\tparent_id\030\002 \001(\004\022\017\n\007proc_id\030\003 \002(\004\022\022" +
-      "\n\nstart_time\030\004 \002(\004\022\r\n\005owner\030\005 \001(\t\022\'\n\005sta" +
-      "te\030\006 \002(\0162\030.hbase.pb.ProcedureState\022\020\n\010st" +
-      "ack_id\030\007 \003(\r\022\023\n\013last_update\030\010 \002(\004\022\017\n\007tim" +
-      "eout\030\t \001(\r\0224\n\texception\030\n \001(\0132!.hbase.pb" +
-      ".ForeignExceptionMessage\022\016\n\006result\030\013 \001(\014" +
-      "\022\022\n\nstate_data\030\014 \001(\014\022\026\n\013nonce_group\030\r \001(" +
-      "\004:\0010\022\020\n\005nonce\030\016 \001(\004:\0010\"+\n\027SequentialProc",
-      "edureData\022\020\n\010executed\030\001 \002(\010\"*\n\031StateMach" +
-      "ineProcedureData\022\r\n\005state\030\001 \003(\r\"X\n\022Proce" +
-      "dureWALHeader\022\017\n\007version\030\001 \002(\r\022\014\n\004type\030\002" +
-      " \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_proc_id\030\004 \002(" +
-      "\004\";\n\023ProcedureWALTrailer\022\017\n\007version\030\001 \002(" +
-      "\r\022\023\n\013tracker_pos\030\002 \002(\004\"\225\001\n\025ProcedureStor" +
-      "eTracker\0229\n\004node\030\001 \003(\0132+.hbase.pb.Proced" +
-      "ureStoreTracker.TrackerNode\032A\n\013TrackerNo" +
-      "de\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(\004\022\017\n" +
-      "\007deleted\030\003 \003(\004\"\257\002\n\021ProcedureWALEntry\022.\n\004",
-      "type\030\001 \002(\0162 .hbase.pb.ProcedureWALEntry." +
-      "Type\022&\n\tprocedure\030\002 \003(\0132\023.hbase.pb.Proce" +
-      "dure\022\017\n\007proc_id\030\003 \001(\004\022\020\n\010child_id\030\004 \003(\004\"" +
-      "\236\001\n\004Type\022\025\n\021PROCEDURE_WAL_EOF\020\001\022\026\n\022PROCE" +
-      "DURE_WAL_INIT\020\002\022\030\n\024PROCEDURE_WAL_INSERT\020" +
-      "\003\022\030\n\024PROCEDURE_WAL_UPDATE\020\004\022\030\n\024PROCEDURE" +
-      "_WAL_DELETE\020\005\022\031\n\025PROCEDURE_WAL_COMPACT\020\006" +
-      "*{\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014\n\010" +
-      "RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_TIMEO" +
-      "UT\020\004\022\016\n\nROLLEDBACK\020\005\022\013\n\007SUCCESS\020\006\022\n\n\006FAI",
-      "LED\020\007BL\n1org.apache.hadoop.hbase.shaded." +
-      "protobuf.generatedB\017ProcedureProtosH\001\210\001\001" +
-      "\240\001\001"
+      "ng.proto\"\317\002\n\tProcedure\022\022\n\nclass_name\030\001 \002" +
+      "(\t\022\021\n\tparent_id\030\002 \001(\004\022\017\n\007proc_id\030\003 \002(\004\022\026" +
+      "\n\016submitted_time\030\004 \002(\004\022\r\n\005owner\030\005 \001(\t\022\'\n" +
+      "\005state\030\006 \002(\0162\030.hbase.pb.ProcedureState\022\020" +
+      "\n\010stack_id\030\007 \003(\r\022\023\n\013last_update\030\010 \002(\004\022\017\n" +
+      "\007timeout\030\t \001(\r\0224\n\texception\030\n \001(\0132!.hbas" +
+      "e.pb.ForeignExceptionMessage\022\016\n\006result\030\013" +
+      " \001(\014\022\022\n\nstate_data\030\014 \001(\014\022\026\n\013nonce_group\030" +
+      "\r \001(\004:\0010\022\020\n\005nonce\030\016 \001(\004:\0010\"+\n\027Sequential",
+      "ProcedureData\022\020\n\010executed\030\001 \002(\010\"*\n\031State" +
+      "MachineProcedureData\022\r\n\005state\030\001 \003(\r\"X\n\022P" +
+      "rocedureWALHeader\022\017\n\007version\030\001 \002(\r\022\014\n\004ty" +
+      "pe\030\002 \002(\r\022\016\n\006log_id\030\003 \002(\004\022\023\n\013min_proc_id\030" +
+      "\004 \002(\004\";\n\023ProcedureWALTrailer\022\017\n\007version\030" +
+      "\001 \002(\r\022\023\n\013tracker_pos\030\002 \002(\004\"\225\001\n\025Procedure" +
+      "StoreTracker\0229\n\004node\030\001 \003(\0132+.hbase.pb.Pr" +
+      "ocedureStoreTracker.TrackerNode\032A\n\013Track" +
+      "erNode\022\020\n\010start_id\030\001 \002(\004\022\017\n\007updated\030\002 \003(" +
+      "\004\022\017\n\007deleted\030\003 \003(\004\"\257\002\n\021ProcedureWALEntry",
+      "\022.\n\004type\030\001 \002(\0162 .hbase.pb.ProcedureWALEn" +
+      "try.Type\022&\n\tprocedure\030\002 \003(\0132\023.hbase.pb.P" +
+      "rocedure\022\017\n\007proc_id\030\003 \001(\004\022\020\n\010child_id\030\004 " +
+      "\003(\004\"\236\001\n\004Type\022\025\n\021PROCEDURE_WAL_EOF\020\001\022\026\n\022P" +
+      "ROCEDURE_WAL_INIT\020\002\022\030\n\024PROCEDURE_WAL_INS" +
+      "ERT\020\003\022\030\n\024PROCEDURE_WAL_UPDATE\020\004\022\030\n\024PROCE" +
+      "DURE_WAL_DELETE\020\005\022\031\n\025PROCEDURE_WAL_COMPA" +
+      "CT\020\006*{\n\016ProcedureState\022\020\n\014INITIALIZING\020\001" +
+      "\022\014\n\010RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_T" +
+      "IMEOUT\020\004\022\016\n\nROLLEDBACK\020\005\022\013\n\007SUCCESS\020\006\022\n\n",
+      "\006FAILED\020\007BL\n1org.apache.hadoop.hbase.sha" +
+      "ded.protobuf.generatedB\017ProcedureProtosH" +
+      "\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -7794,7 +7794,7 @@ public final class ProcedureProtos {
     internal_static_hbase_pb_Procedure_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Procedure_descriptor,
-        new java.lang.String[] { "ClassName", "ParentId", "ProcId", "StartTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", "NonceGroup", "Nonce", });
+        new java.lang.String[] { "ClassName", "ParentId", "ProcId", "SubmittedTime", "Owner", "State", "StackId", "LastUpdate", "Timeout", "Exception", "Result", "StateData", "NonceGroup", "Nonce", });
     internal_static_hbase_pb_SequentialProcedureData_descriptor =
       getDescriptor().getMessageTypes().get(1);
     internal_static_hbase_pb_SequentialProcedureData_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index e22695b..d7d51e2 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -512,7 +512,7 @@ message GetProcedureResultResponse {
   }
 
   required State state = 1;
-  optional uint64 start_time = 2;
+  optional uint64 submitted_time = 2;
   optional uint64 last_update = 3;
   optional bytes result = 4;
   optional ForeignExceptionMessage exception = 5;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
index 2fed24d..1a3ecf5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
@@ -43,7 +43,7 @@ message Procedure {
   required string class_name = 1;        // full classname to be able to instantiate the procedure
   optional uint64 parent_id = 2;         // parent if not a root-procedure otherwise not set
   required uint64 proc_id = 3;
-  required uint64 start_time = 4;
+  required uint64 submitted_time = 4;
   optional string owner = 5;
 
   // internal "runtime" state

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/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 f86f800..9af8f45 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
@@ -1098,7 +1098,7 @@ public class MasterRpcServices extends RSRpcServices
       if (v.getFirst() != null) {
         ProcedureInfo result = v.getFirst();
         builder.setState(GetProcedureResultResponse.State.FINISHED);
-        builder.setStartTime(result.getStartTime());
+        builder.setSubmittedTime(result.getSubmittedTime());
         builder.setLastUpdate(result.getLastUpdate());
         if (result.isFailed()) {
           builder.setException(ForeignExceptionUtil.toProtoForeignException(result.getException()));
@@ -1113,7 +1113,7 @@ public class MasterRpcServices extends RSRpcServices
           builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
         } else {
           builder.setState(GetProcedureResultResponse.State.RUNNING);
-          builder.setStartTime(proc.getStartTime());
+          builder.setSubmittedTime(proc.getSubmittedTime());
           builder.setLastUpdate(proc.getLastUpdate());
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 484decc..2703947 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -166,7 +166,7 @@ implements ServerProcedureInterface {
 
   private void throwProcedureYieldException(final String msg) throws ProcedureYieldException {
     String logMsg = msg + "; cycle=" + this.cycles + ", running for " +
-        StringUtils.formatTimeDiff(System.currentTimeMillis(), getStartTime());
+        StringUtils.formatTimeDiff(System.currentTimeMillis(), getSubmittedTime());
     // The procedure executor logs ProcedureYieldException at trace level. For now, log these
     // yields for server crash processing at DEBUG. Revisit when stable.
     if (LOG.isDebugEnabled()) LOG.debug(logMsg);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/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 645f6fd..c841e61 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
@@ -124,7 +124,7 @@
         <td><%= escapeXml(procInfo.getProcState().toString()) %></a></td>
         <td><%= escapeXml(procInfo.getProcOwner()) %></a></td>
         <td><%= escapeXml(procInfo.getProcName()) %></a></td>
-        <td><%= new Date(procInfo.getStartTime()) %></a></td>
+        <td><%= new Date(procInfo.getSubmittedTime()) %></a></td>
         <td><%= new Date(procInfo.getLastUpdate()) %></a></td>
         <td><%= escapeXml(procInfo.isFailed() ? procInfo.getException().getMessage() : "") %></a></td>
       </tr>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8461456/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb b/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
index 4358a96..64a3388 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_procedures.rb
@@ -29,13 +29,13 @@ EOF
       end
 
       def command()
-        formatter.header([ "Id", "Name", "State", "Start_Time", "Last_Update" ])
+        formatter.header([ "Id", "Name", "State", "Submitted_Time", "Last_Update" ])
 
         list = admin.list_procedures()
         list.each do |proc|
-          start_time = Time.at(proc.getStartTime / 1000).to_s
+          submitted_time = Time.at(proc.getSubmittedTime / 1000).to_s
           last_update = Time.at(proc.getLastUpdate / 1000).to_s
-          formatter.row([ proc.getProcId, proc.getProcName, proc.getProcState, start_time, last_update ])
+          formatter.row([ proc.getProcId, proc.getProcName, proc.getProcState, submitted_time, last_update ])
         end
 
         formatter.footer(list.size)


[44/50] [abbrv] hbase git commit: HBASE-15535 Correct link to Trafodion

Posted by sy...@apache.org.
HBASE-15535 Correct link to Trafodion

Signed-off-by: CHIA-PING TSAI <ch...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: 363f62751c760cc8056a2b1be40a410281e634f7
Parents: 918aa46
Author: G�bor Lipt�k <gl...@gmail.com>
Authored: Sat Apr 15 11:43:38 2017 -0400
Committer: CHIA-PING TSAI <ch...@gmail.com>
Committed: Mon Apr 17 10:26:28 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/sql.adoc      | 2 +-
 src/main/site/xdoc/supportingprojects.xml | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/363f6275/src/main/asciidoc/_chapters/sql.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/sql.adoc b/src/main/asciidoc/_chapters/sql.adoc
index b47104c..b1ad063 100644
--- a/src/main/asciidoc/_chapters/sql.adoc
+++ b/src/main/asciidoc/_chapters/sql.adoc
@@ -37,6 +37,6 @@ link:http://phoenix.apache.org[Apache Phoenix]
 
 === Trafodion
 
-link:https://wiki.trafodion.org/[Trafodion: Transactional SQL-on-HBase]
+link:http://trafodion.incubator.apache.org/[Trafodion: Transactional SQL-on-HBase]
 
 :numbered:

http://git-wip-us.apache.org/repos/asf/hbase/blob/363f6275/src/main/site/xdoc/supportingprojects.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/supportingprojects.xml b/src/main/site/xdoc/supportingprojects.xml
index f349c7f..f949a57 100644
--- a/src/main/site/xdoc/supportingprojects.xml
+++ b/src/main/site/xdoc/supportingprojects.xml
@@ -46,9 +46,9 @@ under the License.
     for HBase.</dd>
    <dt><a href="https://github.com/juwi/HBase-TAggregator">HBase TAggregator</a></dt>
    <dd>An HBase coprocessor for timeseries-based aggregations.</dd>
-   <dt><a href="http://www.trafodion.org">Trafodion</a></dt>
-   <dd>Trafodion is an HP-sponsored Apache-licensed open source SQL on HBase
-    DBMS with full-ACID distributed transaction support.</dd>
+   <dt><a href="http://trafodion.incubator.apache.org/">Apache Trafodion</a></dt>
+   <dd>Apache Trafodion is a webscale SQL-on-Hadoop solution enabling
+    transactional or operational workloads on Hadoop.</dd>
    <dt><a href="http://phoenix.apache.org/">Apache Phoenix</a></dt>
    <dd>Apache Phoenix is a relational database layer over HBase delivered as a
     client-embedded JDBC driver targeting low latency queries over HBase data.</dd>


[29/50] [abbrv] hbase git commit: HBASE-16469 Several log refactoring/improvement suggestions

Posted by sy...@apache.org.
HBASE-16469 Several log refactoring/improvement suggestions

Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 0b5bd78d6e7c51a5c1b6b30a1f385eafcdba8f7b
Parents: 82d554e
Author: Nemo Chen <ch...@gmail.com>
Authored: Wed Apr 5 21:20:40 2017 -0400
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Apr 11 14:16:12 2017 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/client/PreemptiveFastFailInterceptor.java     | 2 +-
 .../test/java/org/apache/hadoop/hbase/HBaseClusterManager.java | 1 +
 .../java/org/apache/hadoop/hbase/regionserver/HRegion.java     | 6 +++---
 .../hadoop/hbase/regionserver/handler/CloseRegionHandler.java  | 2 +-
 .../org/apache/hadoop/hbase/util/MultiThreadedUpdater.java     | 2 +-
 5 files changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0b5bd78d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
index a29a662..abac040 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
@@ -291,7 +291,7 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
     // If we were able to connect to the server, reset the failure
     // information.
     if (couldNotCommunicate == false) {
-      LOG.info("Clearing out PFFE for server " + server.getServerName());
+      LOG.info("Clearing out PFFE for server " + server);
       repeatedFailuresMap.remove(server);
     } else {
       // update time of last attempt

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b5bd78d/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
index 07014e5..d358b9a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
@@ -84,6 +84,7 @@ public class HBaseClusterManager extends Configured implements ClusterManager {
       sshOptions = StringUtils.join(new Object[] { sshOptions, extraSshOptions }, " ");
     }
     sshOptions = (sshOptions == null) ? "" : sshOptions;
+    sshUserName = (sshUserName == null) ? "" : sshUserName;
     tunnelCmd = conf.get("hbase.it.clustermanager.ssh.cmd", DEFAULT_TUNNEL_CMD);
     // Print out ssh special config if any.
     if ((sshUserName != null && sshUserName.length() > 0) ||

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b5bd78d/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 a87b679..78ce608 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
@@ -1390,12 +1390,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   public boolean isMergeable() {
     if (!isAvailable()) {
-      LOG.debug("Region " + getRegionInfo().getRegionNameAsString()
+      LOG.debug("Region " + this
           + " is not mergeable because it is closing or closed");
       return false;
     }
     if (hasReferences()) {
-      LOG.debug("Region " + getRegionInfo().getRegionNameAsString()
+      LOG.debug("Region " + this
           + " is not mergeable because it has references");
       return false;
     }
@@ -1559,7 +1559,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // the close flag?
     if (!abort && worthPreFlushing() && canFlush) {
       status.setStatus("Pre-flushing region before close");
-      LOG.info("Running close preflush of " + getRegionInfo().getRegionNameAsString());
+      LOG.info("Running close preflush of " + this);
       try {
         internalFlushcache(status);
       } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b5bd78d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
index 8aff736..5ff7a1e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
@@ -103,7 +103,7 @@ public class CloseRegionHandler extends EventHandler {
           // This region got closed.  Most likely due to a split.
           // The split message will clean up the master state.
           LOG.warn("Can't close region: was already closed during close(): " +
-            regionInfo.getRegionNameAsString());
+            name);
           return;
         }
       } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b5bd78d/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java
index f41a5cc..564c000 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java
@@ -179,7 +179,7 @@ public class MultiThreadedUpdater extends MultiThreadedWriterBase {
                   LOG.info("Null result expected for the rowkey " + Bytes.toString(rowKey));
                 } else {
                   failedKeySet.add(rowKeyBase);
-                  LOG.error("Failed to update the row with key = [" + rowKey
+                  LOG.error("Failed to update the row with key = [" + Bytes.toString(rowKey)
                       + "], since we could not get the original row");
                 }
               }


[37/50] [abbrv] hbase git commit: HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)

Posted by sy...@apache.org.
HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)


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

Branch: refs/heads/hbase-12439
Commit: 9dd5cda01747ffb91ac084792fa4a8670859e810
Parents: da5fb27
Author: Michael Stack <st...@apache.org>
Authored: Thu Apr 13 21:59:11 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Apr 13 21:59:11 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java   | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9dd5cda0/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index 560ae64..8f56b10 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -432,9 +432,6 @@ public class ThriftServer extends Configured implements Tool {
       throw new RuntimeException("Could not parse the value provided for the port option", e);
     }
 
-    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
-    int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
-
     // Local hostname and user name,
     // used only if QOP is configured.
     String host = null;


[48/50] [abbrv] hbase git commit: HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)

Posted by sy...@apache.org.
HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)


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

Branch: refs/heads/hbase-12439
Commit: c2c2178b2eebe4439eadec6b37fae2566944c16b
Parents: c8cd921
Author: Ramkrishna <ra...@intel.com>
Authored: Mon Apr 17 09:10:59 2017 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Mon Apr 17 09:28:24 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  24 --
 .../org/apache/hadoop/hbase/ExtendedCell.java   |  10 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 +
 .../hbase/regionserver/ByteBufferChunkCell.java |  48 +++
 .../apache/hadoop/hbase/regionserver/Chunk.java |  60 ++-
 .../hadoop/hbase/regionserver/ChunkCreator.java | 404 +++++++++++++++++++
 .../hbase/regionserver/HRegionServer.java       |  14 +-
 .../hbase/regionserver/MemStoreChunkPool.java   | 265 ------------
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |   4 +-
 .../hbase/regionserver/MemStoreLABImpl.java     | 171 ++++----
 .../regionserver/NoTagByteBufferChunkCell.java  |  48 +++
 .../hadoop/hbase/regionserver/OffheapChunk.java |  31 +-
 .../hadoop/hbase/regionserver/OnheapChunk.java  |  32 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   3 +
 .../coprocessor/TestCoprocessorInterface.java   |   4 +
 .../TestRegionObserverScannerOpenHook.java      |   3 +
 .../coprocessor/TestRegionObserverStacking.java |   3 +
 .../io/hfile/TestScannerFromBucketCache.java    |   3 +
 .../hadoop/hbase/master/TestCatalogJanitor.java |   7 +
 .../hadoop/hbase/regionserver/TestBulkLoad.java |   2 +-
 .../hbase/regionserver/TestCellFlatSet.java     |   2 +-
 .../regionserver/TestCompactingMemStore.java    |  37 +-
 .../TestCompactingToCellArrayMapMemStore.java   |  16 +-
 .../TestCompactionArchiveConcurrentClose.java   |   1 +
 .../TestCompactionArchiveIOException.java       |   1 +
 .../regionserver/TestCompactionPolicy.java      |   1 +
 .../hbase/regionserver/TestDefaultMemStore.java |  14 +-
 .../regionserver/TestFailedAppendAndSync.java   |   1 +
 .../hbase/regionserver/TestHMobStore.java       |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   2 +
 .../regionserver/TestHRegionReplayEvents.java   |   2 +-
 .../regionserver/TestMemStoreChunkPool.java     |  48 +--
 .../hbase/regionserver/TestMemStoreLAB.java     |  27 +-
 .../TestMemstoreLABWithoutPool.java             | 168 ++++++++
 .../hbase/regionserver/TestRecoveredEdits.java  |   1 +
 .../hbase/regionserver/TestRegionIncrement.java |   1 +
 .../hadoop/hbase/regionserver/TestStore.java    |   1 +
 .../TestStoreFileRefresherChore.java            |   1 +
 .../hbase/regionserver/TestWALLockup.java       |   1 +
 .../TestWALMonotonicallyIncreasingSeqId.java    |   1 +
 .../hbase/regionserver/wal/TestDurability.java  |   3 +
 41 files changed, 990 insertions(+), 479 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index e1bc969..56de21b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -3135,28 +3135,4 @@ public final class CellUtil {
       return Type.DeleteFamily.getCode();
     }
   }
-
-  /**
-   * Clone the passed cell by copying its data into the passed buf.
-   */
-  public static Cell copyCellTo(Cell cell, ByteBuffer buf, int offset, int len) {
-    int tagsLen = cell.getTagsLength();
-    if (cell instanceof ExtendedCell) {
-      ((ExtendedCell) cell).write(buf, offset);
-    } else {
-      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
-      // other case also. The data fragments within Cell is copied into buf as in KeyValue
-      // serialization format only.
-      KeyValueUtil.appendTo(cell, buf, offset, true);
-    }
-    if (tagsLen == 0) {
-      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
-      // which directly return tagsLen as 0. So we avoid parsing many length components in
-      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
-      // call getTagsLength().
-      return new NoTagsByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
-    } else {
-      return new ByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
index 517873f..10f20ca 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.io.HeapSize;
 public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestamp, HeapSize,
     Cloneable {
 
+  public static int CELL_NOT_BASED_ON_CHUNK = -1;
   /**
    * Write this cell to an OutputStream in a {@link KeyValue} format.
    * <br> KeyValue format <br>
@@ -73,4 +74,13 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
    * @return The deep cloned cell
    */
   Cell deepClone();
+
+  /**
+   * Extracts the id of the backing bytebuffer of this cell if it was obtained from fixed sized
+   * chunks as in case of MemstoreLAB
+   * @return the chunk id if the cell is backed by fixed sized Chunks, else return -1
+   */
+  default int getChunkId() {
+    return CELL_NOT_BASED_ON_CHUNK;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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 bb9f282..f9670e1 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
@@ -748,6 +748,8 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
+    // Initialize the chunkCreator
+    initializeMemStoreChunkCreator();
     this.fileSystemManager = new MasterFileSystem(this);
     this.walManager = new MasterWalManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
new file mode 100644
index 0000000..a8f1000
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
@@ -0,0 +1,48 @@
+/**
+ * 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 java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset
+ * @see MemStoreLAB
+ */
+//TODO : When moving this cell to CellChunkMap we will have the following things
+// to be serialized
+// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
+@InterfaceAudience.Private
+public class ByteBufferChunkCell extends ByteBufferKeyValue {
+  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
index 2cbf0a3..fc4aa0b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
@@ -21,8 +21,10 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /**
  * A chunk of memory out of which allocations are sliced.
@@ -46,13 +48,41 @@ public abstract class Chunk {
   /** Size of chunk in bytes */
   protected final int size;
 
+  // The unique id associated with the chunk.
+  private final int id;
+
+  // indicates if the chunk is formed by ChunkCreator#MemstorePool
+  private final boolean fromPool;
+
+  /**
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so
+   * this is cheap.
+   * @param size in bytes
+   * @param id the chunk id
+   */
+  public Chunk(int size, int id) {
+    this(size, id, false);
+  }
+
   /**
-   * Create an uninitialized chunk. Note that memory is not allocated yet, so this is cheap.
-   *
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so
+   * this is cheap.
    * @param size in bytes
+   * @param id the chunk id
+   * @param fromPool if the chunk is formed by pool
    */
-  Chunk(int size) {
+  public Chunk(int size, int id, boolean fromPool) {
     this.size = size;
+    this.id = id;
+    this.fromPool = fromPool;
+  }
+
+  int getId() {
+    return this.id;
+  }
+
+  boolean isFromPool() {
+    return this.fromPool;
   }
 
   /**
@@ -60,7 +90,24 @@ public abstract class Chunk {
    * constructed the chunk. It is thread-safe against other threads calling alloc(), who will block
    * until the allocation is complete.
    */
-  public abstract void init();
+  public void init() {
+    assert nextFreeOffset.get() == UNINITIALIZED;
+    try {
+      allocateDataBuffer();
+    } catch (OutOfMemoryError e) {
+      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
+      assert failInit; // should be true.
+      throw e;
+    }
+    // Mark that it's ready for use
+    // Move 8 bytes since the first 8 bytes are having the chunkid in it
+    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, Bytes.SIZEOF_LONG);
+    // We should always succeed the above CAS since only one thread
+    // calls init()!
+    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
+  }
+
+  abstract void allocateDataBuffer();
 
   /**
    * Reset the offset to UNINITIALIZED before before reusing an old chunk
@@ -74,7 +121,8 @@ public abstract class Chunk {
 
   /**
    * Try to allocate <code>size</code> bytes from the chunk.
-   *
+   * If a chunk is tried to get allocated before init() call, the thread doing the allocation
+   * will be in busy-wait state as it will keep looping till the nextFreeOffset is set.
    * @return the offset of the successful allocation, or -1 to indicate not-enough-space
    */
   public int alloc(int size) {
@@ -96,7 +144,7 @@ public abstract class Chunk {
       if (oldOffset + size > data.capacity()) {
         return -1; // alloc doesn't fit
       }
-
+      // TODO : If seqID is to be written add 8 bytes here for nextFreeOFfset
       // Try to atomically claim this chunk
       if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size)) {
         // we got the alloc

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
new file mode 100644
index 0000000..073fb25
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -0,0 +1,404 @@
+
+/**
+ * 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 java.lang.ref.SoftReference;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Does the management of memstoreLAB chunk creations. A monotonically incrementing id is associated
+ * with every chunk
+ */
+@InterfaceAudience.Private
+public class ChunkCreator {
+  private final Log LOG = LogFactory.getLog(ChunkCreator.class);
+  // monotonically increasing chunkid
+  private AtomicInteger chunkID = new AtomicInteger(1);
+  // maps the chunk against the monotonically increasing chunk id. We need to preserve the
+  // natural ordering of the key
+  // CellChunkMap creation should convert the soft ref to hard reference
+  private Map<Integer, SoftReference<Chunk>> chunkIdMap =
+      new ConcurrentHashMap<Integer, SoftReference<Chunk>>();
+  private final int chunkSize;
+  private final boolean offheap;
+  @VisibleForTesting
+  static ChunkCreator INSTANCE;
+  @VisibleForTesting
+  static boolean chunkPoolDisabled = false;
+  private MemStoreChunkPool pool;
+
+  @VisibleForTesting
+  ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
+      float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
+    this.chunkSize = chunkSize;
+    this.offheap = offheap;
+    this.pool = initializePool(globalMemStoreSize, poolSizePercentage, initialCountPercentage);
+    if (heapMemoryManager != null && this.pool != null) {
+      // Register with Heap Memory manager
+      heapMemoryManager.registerTuneObserver(this.pool);
+    }
+  }
+
+  /**
+   * Initializes the instance of MSLABChunkCreator
+   * @param chunkSize the chunkSize
+   * @param offheap indicates if the chunk is to be created offheap or not
+   * @param globalMemStoreSize  the global memstore size
+   * @param poolSizePercentage pool size percentage
+   * @param initialCountPercentage the initial count of the chunk pool if any
+   * @param heapMemoryManager the heapmemory manager
+   * @return singleton MSLABChunkCreator
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
+      justification = "Method is called by single thread at the starting of RS")
+  @VisibleForTesting
+  public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
+      float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
+    if (INSTANCE != null) return INSTANCE;
+    INSTANCE = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
+        initialCountPercentage, heapMemoryManager);
+    return INSTANCE;
+  }
+
+  static ChunkCreator getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Creates and inits a chunk.
+   * @return the chunk that was initialized
+   */
+  Chunk getChunk() {
+    Chunk chunk = null;
+    if (pool != null) {
+      //  the pool creates the chunk internally. The chunk#init() call happens here
+      chunk = this.pool.getChunk();
+      // the pool has run out of maxCount
+      if (chunk == null) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("The chunk pool is full. Reached maxCount= " + this.pool.getMaxCount()
+              + ". Creating chunk onheap.");
+        }
+      }
+    }
+    if (chunk == null) {
+      chunk = createChunk();
+    }
+    // put this chunk into the chunkIdMap
+    this.chunkIdMap.put(chunk.getId(), new SoftReference<>(chunk));
+    // now we need to actually do the expensive memory allocation step in case of a new chunk,
+    // else only the offset is set to the beginning of the chunk to accept allocations
+    chunk.init();
+    return chunk;
+  }
+
+  private Chunk createChunk() {
+    return createChunk(false);
+  }
+
+  /**
+   * Creates the chunk either onheap or offheap
+   * @param pool indicates if the chunks have to be created which will be used by the Pool
+   * @return the chunk
+   */
+  private Chunk createChunk(boolean pool) {
+    int id = chunkID.getAndIncrement();
+    assert id > 0;
+    // do not create offheap chunk on demand
+    if (pool && this.offheap) {
+      return new OffheapChunk(chunkSize, id, pool);
+    } else {
+      return new OnheapChunk(chunkSize, id, pool);
+    }
+  }
+
+  @VisibleForTesting
+  // TODO : To be used by CellChunkMap
+  Chunk getChunk(int id) {
+    SoftReference<Chunk> ref = chunkIdMap.get(id);
+    if (ref != null) {
+      return ref.get();
+    }
+    return null;
+  }
+
+  int getChunkSize() {
+    return this.chunkSize;
+  }
+
+  boolean isOffheap() {
+    return this.offheap;
+  }
+
+  private void removeChunks(Set<Integer> chunkIDs) {
+    this.chunkIdMap.keySet().removeAll(chunkIDs);
+  }
+
+  Chunk removeChunk(int chunkId) {
+    SoftReference<Chunk> ref = this.chunkIdMap.remove(chunkId);
+    if (ref != null) {
+      return ref.get();
+    }
+    return null;
+  }
+
+  @VisibleForTesting
+  int size() {
+    return this.chunkIdMap.size();
+  }
+
+  @VisibleForTesting
+  void clearChunkIds() {
+    this.chunkIdMap.clear();
+  }
+
+  /**
+   * A pool of {@link Chunk} instances.
+   *
+   * MemStoreChunkPool caches a number of retired chunks for reusing, it could
+   * decrease allocating bytes when writing, thereby optimizing the garbage
+   * collection on JVM.
+   */
+  private  class MemStoreChunkPool implements HeapMemoryTuneObserver {
+    private int maxCount;
+
+    // A queue of reclaimed chunks
+    private final BlockingQueue<Chunk> reclaimedChunks;
+    private final float poolSizePercentage;
+
+    /** Statistics thread schedule pool */
+    private final ScheduledExecutorService scheduleThreadPool;
+    /** Statistics thread */
+    private static final int statThreadPeriod = 60 * 5;
+    private final AtomicLong chunkCount = new AtomicLong();
+    private final AtomicLong reusedChunkCount = new AtomicLong();
+
+    MemStoreChunkPool(int maxCount, int initialCount, float poolSizePercentage) {
+      this.maxCount = maxCount;
+      this.poolSizePercentage = poolSizePercentage;
+      this.reclaimedChunks = new LinkedBlockingQueue<>();
+      for (int i = 0; i < initialCount; i++) {
+        Chunk chunk = createChunk(true);
+        chunk.init();
+        reclaimedChunks.add(chunk);
+      }
+      chunkCount.set(initialCount);
+      final String n = Thread.currentThread().getName();
+      scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
+          .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
+      this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
+          statThreadPeriod, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
+     * not yet created max allowed chunks count. When we have already created max allowed chunks and
+     * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
+     * then.
+     * Note: Chunks returned by this pool must be put back to the pool after its use.
+     * @return a chunk
+     * @see #putbackChunks(Set)
+     */
+    Chunk getChunk() {
+      Chunk chunk = reclaimedChunks.poll();
+      if (chunk != null) {
+        chunk.reset();
+        reusedChunkCount.incrementAndGet();
+      } else {
+        // Make a chunk iff we have not yet created the maxCount chunks
+        while (true) {
+          long created = this.chunkCount.get();
+          if (created < this.maxCount) {
+            if (this.chunkCount.compareAndSet(created, created + 1)) {
+              chunk = createChunk(true);
+              break;
+            }
+          } else {
+            break;
+          }
+        }
+      }
+      return chunk;
+    }
+
+    /**
+     * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
+     * chunks
+     * @param chunks
+     */
+    private void putbackChunks(Set<Integer> chunks) {
+      int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
+      Iterator<Integer> iterator = chunks.iterator();
+      while (iterator.hasNext()) {
+        Integer chunkId = iterator.next();
+        // remove the chunks every time though they are from the pool or not
+        Chunk chunk = ChunkCreator.this.removeChunk(chunkId);
+        if (chunk != null) {
+          if (chunk.isFromPool() && toAdd > 0) {
+            reclaimedChunks.add(chunk);
+          }
+          toAdd--;
+        }
+      }
+    }
+
+    private class StatisticsThread extends Thread {
+      StatisticsThread() {
+        super("MemStoreChunkPool.StatisticsThread");
+        setDaemon(true);
+      }
+
+      @Override
+      public void run() {
+        logStats();
+      }
+
+      private void logStats() {
+        if (!LOG.isDebugEnabled()) return;
+        long created = chunkCount.get();
+        long reused = reusedChunkCount.get();
+        long total = created + reused;
+        LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
+            + ",created chunk count=" + created
+            + ",reused chunk count=" + reused
+            + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
+                (float) reused / (float) total, 2)));
+      }
+    }
+
+    private int getMaxCount() {
+      return this.maxCount;
+    }
+
+    @Override
+    public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
+      // don't do any tuning in case of offheap memstore
+      if (isOffheap()) {
+        LOG.warn("Not tuning the chunk pool as it is offheap");
+        return;
+      }
+      int newMaxCount =
+          (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
+      if (newMaxCount != this.maxCount) {
+        // We need an adjustment in the chunks numbers
+        if (newMaxCount > this.maxCount) {
+          // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
+          // create and add them to Q
+          LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
+          this.maxCount = newMaxCount;
+        } else {
+          // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
+          // itself. If the extra chunks are serving already, do not pool those when we get them back
+          LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
+          this.maxCount = newMaxCount;
+          if (this.reclaimedChunks.size() > newMaxCount) {
+            synchronized (this) {
+              while (this.reclaimedChunks.size() > newMaxCount) {
+                this.reclaimedChunks.poll();
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  static void clearDisableFlag() {
+    chunkPoolDisabled = false;
+  }
+
+  private MemStoreChunkPool initializePool(long globalMemStoreSize, float poolSizePercentage,
+      float initialCountPercentage) {
+    if (poolSizePercentage <= 0) {
+      LOG.info("PoolSizePercentage is less than 0. So not using pool");
+      return null;
+    }
+    if (chunkPoolDisabled) {
+      return null;
+    }
+    if (poolSizePercentage > 1.0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / getChunkSize());
+    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int initialCount = (int) (initialCountPercentage * maxCount);
+    LOG.info("Allocating MemStoreChunkPool with chunk size "
+        + StringUtils.byteDesc(getChunkSize()) + ", max count " + maxCount
+        + ", initial count " + initialCount);
+    return new MemStoreChunkPool(maxCount, initialCount, poolSizePercentage);
+  }
+
+  @VisibleForTesting
+  int getMaxCount() {
+    if (pool != null) {
+      return pool.getMaxCount();
+    }
+    return 0;
+  }
+
+  @VisibleForTesting
+  int getPoolSize() {
+    if (pool != null) {
+      return pool.reclaimedChunks.size();
+    }
+    return 0;
+  }
+
+  /*
+   * Only used in testing
+   */
+  @VisibleForTesting
+  void clearChunksInPool() {
+    if (pool != null) {
+      pool.reclaimedChunks.clear();
+    }
+  }
+
+  synchronized void putbackChunks(Set<Integer> chunks) {
+    if (pool != null) {
+      pool.putbackChunks(chunks);
+    } else {
+      this.removeChunks(chunks);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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 b3b5113..41eb0a3 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
@@ -1472,7 +1472,7 @@ public class HRegionServer extends HasThread implements
       startServiceThreads();
       startHeapMemoryManager();
       // Call it after starting HeapMemoryManager.
-      initializeMemStoreChunkPool();
+      initializeMemStoreChunkCreator();
       LOG.info("Serving as " + this.serverName +
         ", RpcServer on " + rpcServices.isa +
         ", sessionid=0x" +
@@ -1492,7 +1492,7 @@ public class HRegionServer extends HasThread implements
     }
   }
 
-  private void initializeMemStoreChunkPool() {
+  protected void initializeMemStoreChunkCreator() {
     if (MemStoreLAB.isEnabled(conf)) {
       // MSLAB is enabled. So initialize MemStoreChunkPool
       // By this time, the MemstoreFlusher is already initialized. We can get the global limits from
@@ -1506,12 +1506,10 @@ public class HRegionServer extends HasThread implements
       float initialCountPercentage = conf.getFloat(MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY,
           MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT);
       int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY, MemStoreLAB.CHUNK_SIZE_DEFAULT);
-      MemStoreChunkPool pool = MemStoreChunkPool.initialize(globalMemStoreSize, poolSizePercentage,
-          initialCountPercentage, chunkSize, offheap);
-      if (pool != null && this.hMemManager != null) {
-        // Register with Heap Memory manager
-        this.hMemManager.registerTuneObserver(pool);
-      }
+      // init the chunkCreator
+      ChunkCreator chunkCreator =
+          ChunkCreator.initialize(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
+            initialCountPercentage, this.hMemManager);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
deleted file mode 100644
index b7ac212..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * A pool of {@link Chunk} instances.
- * 
- * MemStoreChunkPool caches a number of retired chunks for reusing, it could
- * decrease allocating bytes when writing, thereby optimizing the garbage
- * collection on JVM.
- * 
- * The pool instance is globally unique and could be obtained through
- * {@link MemStoreChunkPool#initialize(long, float, float, int, boolean)}
- * 
- * {@link MemStoreChunkPool#getChunk()} is called when MemStoreLAB allocating
- * bytes, and {@link MemStoreChunkPool#putbackChunks(BlockingQueue)} is called
- * when MemStore clearing snapshot for flush
- */
-@SuppressWarnings("javadoc")
-@InterfaceAudience.Private
-public class MemStoreChunkPool implements HeapMemoryTuneObserver {
-  private static final Log LOG = LogFactory.getLog(MemStoreChunkPool.class);
-
-  // Static reference to the MemStoreChunkPool
-  static MemStoreChunkPool GLOBAL_INSTANCE;
-  /** Boolean whether we have disabled the memstore chunk pool entirely. */
-  static boolean chunkPoolDisabled = false;
-
-  private int maxCount;
-
-  // A queue of reclaimed chunks
-  private final BlockingQueue<Chunk> reclaimedChunks;
-  private final int chunkSize;
-  private final float poolSizePercentage;
-
-  /** Statistics thread schedule pool */
-  private final ScheduledExecutorService scheduleThreadPool;
-  /** Statistics thread */
-  private static final int statThreadPeriod = 60 * 5;
-  private final AtomicLong chunkCount = new AtomicLong();
-  private final AtomicLong reusedChunkCount = new AtomicLong();
-  private final boolean offheap;
-
-  MemStoreChunkPool(int chunkSize, int maxCount, int initialCount, float poolSizePercentage,
-      boolean offheap) {
-    this.maxCount = maxCount;
-    this.chunkSize = chunkSize;
-    this.poolSizePercentage = poolSizePercentage;
-    this.offheap = offheap;
-    this.reclaimedChunks = new LinkedBlockingQueue<>();
-    for (int i = 0; i < initialCount; i++) {
-      Chunk chunk = this.offheap ? new OffheapChunk(chunkSize) : new OnheapChunk(chunkSize);
-      chunk.init();
-      reclaimedChunks.add(chunk);
-    }
-    chunkCount.set(initialCount);
-    final String n = Thread.currentThread().getName();
-    scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
-        .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
-    this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
-        statThreadPeriod, TimeUnit.SECONDS);
-  }
-
-  /**
-   * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
-   * not yet created max allowed chunks count. When we have already created max allowed chunks and
-   * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
-   * then.
-   * Note: Chunks returned by this pool must be put back to the pool after its use.
-   * @return a chunk
-   * @see #putbackChunk(Chunk)
-   * @see #putbackChunks(BlockingQueue)
-   */
-  Chunk getChunk() {
-    Chunk chunk = reclaimedChunks.poll();
-    if (chunk != null) {
-      chunk.reset();
-      reusedChunkCount.incrementAndGet();
-    } else {
-      // Make a chunk iff we have not yet created the maxCount chunks
-      while (true) {
-        long created = this.chunkCount.get();
-        if (created < this.maxCount) {
-          chunk = this.offheap ? new OffheapChunk(this.chunkSize) : new OnheapChunk(this.chunkSize);
-          if (this.chunkCount.compareAndSet(created, created + 1)) {
-            break;
-          }
-        } else {
-          break;
-        }
-      }
-    }
-    return chunk;
-  }
-
-  /**
-   * Add the chunks to the pool, when the pool achieves the max size, it will
-   * skip the remaining chunks
-   * @param chunks
-   */
-  synchronized void putbackChunks(BlockingQueue<Chunk> chunks) {
-    int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
-    Chunk chunk = null;
-    while ((chunk = chunks.poll()) != null && toAdd > 0) {
-      reclaimedChunks.add(chunk);
-      toAdd--;
-    }
-  }
-
-  /**
-   * Add the chunk to the pool, if the pool has achieved the max size, it will
-   * skip it
-   * @param chunk
-   */
-  synchronized void putbackChunk(Chunk chunk) {
-    if (reclaimedChunks.size() < this.maxCount) {
-      reclaimedChunks.add(chunk);
-    }
-  }
-
-  int getPoolSize() {
-    return this.reclaimedChunks.size();
-  }
-
-  /*
-   * Only used in testing
-   */
-  void clearChunks() {
-    this.reclaimedChunks.clear();
-  }
-
-  private class StatisticsThread extends Thread {
-    StatisticsThread() {
-      super("MemStoreChunkPool.StatisticsThread");
-      setDaemon(true);
-    }
-
-    @Override
-    public void run() {
-      logStats();
-    }
-
-    private void logStats() {
-      if (!LOG.isDebugEnabled()) return;
-      long created = chunkCount.get();
-      long reused = reusedChunkCount.get();
-      long total = created + reused;
-      LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
-          + ",created chunk count=" + created
-          + ",reused chunk count=" + reused
-          + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
-              (float) reused / (float) total, 2)));
-    }
-  }
-
-  /**
-   * @return the global MemStoreChunkPool instance
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
-      justification = "Method is called by single thread at the starting of RS")
-  static MemStoreChunkPool initialize(long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage, int chunkSize, boolean offheap) {
-    if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
-    if (chunkPoolDisabled) return null;
-
-    if (poolSizePercentage <= 0) {
-      chunkPoolDisabled = true;
-      return null;
-    }
-    if (poolSizePercentage > 1.0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
-    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int initialCount = (int) (initialCountPercentage * maxCount);
-    LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
-        + ", max count " + maxCount + ", initial count " + initialCount);
-    GLOBAL_INSTANCE = new MemStoreChunkPool(chunkSize, maxCount, initialCount, poolSizePercentage,
-        offheap);
-    return GLOBAL_INSTANCE;
-  }
-
-  /**
-   * @return The singleton instance of this pool.
-   */
-  static MemStoreChunkPool getPool() {
-    return GLOBAL_INSTANCE;
-  }
-
-  int getMaxCount() {
-    return this.maxCount;
-  }
-
-  @VisibleForTesting
-  static void clearDisableFlag() {
-    chunkPoolDisabled = false;
-  }
-
-  @Override
-  public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
-    // don't do any tuning in case of offheap memstore
-    if (this.offheap) {
-      LOG.warn("Not tuning the chunk pool as it is offheap");
-      return;
-    }
-    int newMaxCount = (int) (newMemstoreSize * poolSizePercentage / chunkSize);
-    if (newMaxCount != this.maxCount) {
-      // We need an adjustment in the chunks numbers
-      if (newMaxCount > this.maxCount) {
-        // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
-        // create and add them to Q
-        LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
-        this.maxCount = newMaxCount;
-      } else {
-        // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
-        // itself. If the extra chunks are serving already, do not pool those when we get them back
-        LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
-        this.maxCount = newMaxCount;
-        if (this.reclaimedChunks.size() > newMaxCount) {
-          synchronized (this) {
-            while (this.reclaimedChunks.size() > newMaxCount) {
-              this.reclaimedChunks.poll();
-            }
-          }
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
index f6d1607..72e937c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * <p>
  * The MemStoreLAB is basically a bump-the-pointer allocator that allocates big (2MB) chunks from
  * and then doles it out to threads that request slices into the array. These chunks can get pooled
- * as well. See {@link MemStoreChunkPool}.
+ * as well. See {@link ChunkCreator}.
  * <p>
  * The purpose of this is to combat heap fragmentation in the regionserver. By ensuring that all
  * Cells in a given memstore refer only to large chunks of contiguous memory, we ensure that
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * {@link #copyCellInto(Cell)} gets called. This allocates enough size in the chunk to hold this
  * cell's data and copies into this area and then recreate a Cell over this copied data.
  * <p>
- * @see MemStoreChunkPool
+ * @see ChunkCreator
  */
 @InterfaceAudience.Private
 public interface MemStoreLAB {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 4e87135..4fba82d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -18,23 +18,26 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.nio.ByteBuffer;
+import java.util.Set;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
 
 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.ExtendedCell;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-
 /**
  * A memstore-local allocation buffer.
  * <p>
@@ -55,8 +58,8 @@ import com.google.common.base.Preconditions;
  * would provide a performance improvement - probably would speed up the
  * Bytes.toLong/Bytes.toInt calls in KeyValue, but some of those are cached
  * anyway.
- * The chunks created by this MemStoreLAB can get pooled at {@link MemStoreChunkPool}.
- * When the Chunk comes pool, it can be either an on heap or an off heap backed chunk. The chunks,
+ * The chunks created by this MemStoreLAB can get pooled at {@link ChunkCreator}.
+ * When the Chunk comes from pool, it can be either an on heap or an off heap backed chunk. The chunks,
  * which this MemStoreLAB creates on its own (when no chunk available from pool), those will be
  * always on heap backed.
  */
@@ -66,14 +69,15 @@ public class MemStoreLABImpl implements MemStoreLAB {
   static final Log LOG = LogFactory.getLog(MemStoreLABImpl.class);
 
   private AtomicReference<Chunk> curChunk = new AtomicReference<>();
-  // A queue of chunks from pool contained by this memstore LAB
-  // TODO: in the future, it would be better to have List implementation instead of Queue,
-  // as FIFO order is not so important here
+  // Lock to manage multiple handlers requesting for a chunk
+  private ReentrantLock lock = new ReentrantLock();
+
+  // A set of chunks contained by this memstore LAB
   @VisibleForTesting
-  BlockingQueue<Chunk> pooledChunkQueue = null;
+  Set<Integer> chunks = new ConcurrentSkipListSet<Integer>();
   private final int chunkSize;
   private final int maxAlloc;
-  private final MemStoreChunkPool chunkPool;
+  private final ChunkCreator chunkCreator;
 
   // This flag is for closing this instance, its set when clearing snapshot of
   // memstore
@@ -92,20 +96,12 @@ public class MemStoreLABImpl implements MemStoreLAB {
   public MemStoreLABImpl(Configuration conf) {
     chunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);
     maxAlloc = conf.getInt(MAX_ALLOC_KEY, MAX_ALLOC_DEFAULT);
-    this.chunkPool = MemStoreChunkPool.getPool();
-    // currently chunkQueue is only used for chunkPool
-    if (this.chunkPool != null) {
-      // set queue length to chunk pool max count to avoid keeping reference of
-      // too many non-reclaimable chunks
-      pooledChunkQueue = new LinkedBlockingQueue<>(chunkPool.getMaxCount());
-    }
-
+    this.chunkCreator = ChunkCreator.getInstance();
     // if we don't exclude allocations >CHUNK_SIZE, we'd infiniteloop on one!
     Preconditions.checkArgument(maxAlloc <= chunkSize,
         MAX_ALLOC_KEY + " must be less than " + CHUNK_SIZE_KEY);
   }
 
-
   @Override
   public Cell copyCellInto(Cell cell) {
     int size = KeyValueUtil.length(cell);
@@ -118,19 +114,52 @@ public class MemStoreLABImpl implements MemStoreLAB {
     Chunk c = null;
     int allocOffset = 0;
     while (true) {
+      // Try to get the chunk
       c = getOrMakeChunk();
+      // we may get null because the some other thread succeeded in getting the lock
+      // and so the current thread has to try again to make its chunk or grab the chunk
+      // that the other thread created
       // Try to allocate from this chunk
-      allocOffset = c.alloc(size);
-      if (allocOffset != -1) {
-        // We succeeded - this is the common case - small alloc
-        // from a big buffer
-        break;
+      if (c != null) {
+        allocOffset = c.alloc(size);
+        if (allocOffset != -1) {
+          // We succeeded - this is the common case - small alloc
+          // from a big buffer
+          break;
+        }
+        // not enough space!
+        // try to retire this chunk
+        tryRetireChunk(c);
       }
-      // not enough space!
-      // try to retire this chunk
-      tryRetireChunk(c);
     }
-    return CellUtil.copyCellTo(cell, c.getData(), allocOffset, size);
+    return copyToChunkCell(cell, c.getData(), allocOffset, size);
+  }
+
+  /**
+   * Clone the passed cell by copying its data into the passed buf and create a cell with a chunkid
+   * out of it
+   */
+  private Cell copyToChunkCell(Cell cell, ByteBuffer buf, int offset, int len) {
+    int tagsLen = cell.getTagsLength();
+    if (cell instanceof ExtendedCell) {
+      ((ExtendedCell) cell).write(buf, offset);
+    } else {
+      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
+      // other case also. The data fragments within Cell is copied into buf as in KeyValue
+      // serialization format only.
+      KeyValueUtil.appendTo(cell, buf, offset, true);
+    }
+    // TODO : write the seqid here. For writing seqId we should create a new cell type so
+    // that seqId is not used as the state
+    if (tagsLen == 0) {
+      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
+      // which directly return tagsLen as 0. So we avoid parsing many length components in
+      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
+      // call getTagsLength().
+      return new NoTagByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+    } else {
+      return new ByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+    }
   }
 
   /**
@@ -142,9 +171,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
     this.closed = true;
     // We could put back the chunks to pool for reusing only when there is no
     // opening scanner which will read their data
-    if (chunkPool != null && openScannerCount.get() == 0
-        && reclaimed.compareAndSet(false, true)) {
-      chunkPool.putbackChunks(this.pooledChunkQueue);
+    int count  = openScannerCount.get();
+    if(count == 0) {
+      recycleChunks();
     }
   }
 
@@ -162,9 +191,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
   @Override
   public void decScannerCount() {
     int count = this.openScannerCount.decrementAndGet();
-    if (this.closed && chunkPool != null && count == 0
-        && reclaimed.compareAndSet(false, true)) {
-      chunkPool.putbackChunks(this.pooledChunkQueue);
+    if (this.closed && count == 0) {
+      recycleChunks();
+    }
+  }
+
+  private void recycleChunks() {
+    if (reclaimed.compareAndSet(false, true)) {
+      chunkCreator.putbackChunks(chunks);
     }
   }
 
@@ -190,45 +224,33 @@ public class MemStoreLABImpl implements MemStoreLAB {
    * allocate a new one from the JVM.
    */
   private Chunk getOrMakeChunk() {
-    while (true) {
-      // Try to get the chunk
-      Chunk c = curChunk.get();
-      if (c != null) {
-        return c;
-      }
-
-      // No current chunk, so we want to allocate one. We race
-      // against other allocators to CAS in an uninitialized chunk
-      // (which is cheap to allocate)
-      if (chunkPool != null) {
-        c = chunkPool.getChunk();
-      }
-      boolean pooledChunk = false;
-      if (c != null) {
-        // This is chunk from pool
-        pooledChunk = true;
-      } else {
-        c = new OnheapChunk(chunkSize);// When chunk is not from pool, always make it as on heap.
-      }
-      if (curChunk.compareAndSet(null, c)) {
-        // we won race - now we need to actually do the expensive
-        // allocation step
-        c.init();
-        if (pooledChunk) {
-          if (!this.closed && !this.pooledChunkQueue.offer(c)) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Chunk queue is full, won't reuse this new chunk. Current queue size: "
-                  + pooledChunkQueue.size());
-            }
-          }
+    // Try to get the chunk
+    Chunk c = curChunk.get();
+    if (c != null) {
+      return c;
+    }
+    // No current chunk, so we want to allocate one. We race
+    // against other allocators to CAS in an uninitialized chunk
+    // (which is cheap to allocate)
+    if (lock.tryLock()) {
+      try {
+        // once again check inside the lock
+        c = curChunk.get();
+        if (c != null) {
+          return c;
         }
-        return c;
-      } else if (pooledChunk) {
-        chunkPool.putbackChunk(c);
+        c = this.chunkCreator.getChunk();
+        if (c != null) {
+          // set the curChunk. No need of CAS as only one thread will be here
+          curChunk.set(c);
+          chunks.add(c.getId());
+          return c;
+        }
+      } finally {
+        lock.unlock();
       }
-      // someone else won race - that's fine, we'll try to grab theirs
-      // in the next iteration of the loop.
     }
+    return null;
   }
 
   @VisibleForTesting
@@ -236,8 +258,15 @@ public class MemStoreLABImpl implements MemStoreLAB {
     return this.curChunk.get();
   }
 
-
+  @VisibleForTesting
   BlockingQueue<Chunk> getPooledChunks() {
-    return this.pooledChunkQueue;
+    BlockingQueue<Chunk> pooledChunks = new LinkedBlockingQueue<>();
+    for (Integer id : this.chunks) {
+      Chunk chunk = chunkCreator.getChunk(id);
+      if (chunk != null && chunk.isFromPool()) {
+        pooledChunks.add(chunk);
+      }
+    }
+    return pooledChunks;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
new file mode 100644
index 0000000..a8ba50c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
@@ -0,0 +1,48 @@
+/**
+ * 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 java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
+ * @see MemStoreLAB
+ */
+@InterfaceAudience.Private
+public class NoTagByteBufferChunkCell extends NoTagsByteBufferKeyValue {
+
+  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
index ed98cfa..e244a33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
@@ -21,34 +21,27 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
-import com.google.common.base.Preconditions;
-
 /**
  * An off heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OffheapChunk extends Chunk {
 
-  OffheapChunk(int size) {
-    super(size);
+  OffheapChunk(int size, int id) {
+    // better if this is always created fromPool. This should not be called
+    super(size, id);
+  }
+
+  OffheapChunk(int size, int id, boolean fromPool) {
+    super(size, id, fromPool);
+    assert fromPool == true;
   }
 
   @Override
-  public void init() {
-    assert nextFreeOffset.get() == UNINITIALIZED;
-    try {
-      if (data == null) {
-        data = ByteBuffer.allocateDirect(this.size);
-      }
-    } catch (OutOfMemoryError e) {
-      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
-      assert failInit; // should be true.
-      throw e;
+  void allocateDataBuffer() {
+    if (data == null) {
+      data = ByteBuffer.allocateDirect(this.size);
+      data.putLong(0, this.getId());
     }
-    // Mark that it's ready for use
-    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
-    // We should always succeed the above CAS since only one thread
-    // calls init()!
-    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
index bd33cb5..da34e24 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
@@ -21,33 +21,25 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
-import com.google.common.base.Preconditions;
-
 /**
  * An on heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OnheapChunk extends Chunk {
 
-  OnheapChunk(int size) {
-    super(size);
+  OnheapChunk(int size, int id) {
+    super(size, id);
+  }
+
+  OnheapChunk(int size, int id, boolean fromPool) {
+    super(size, id, fromPool);
   }
 
-  public void init() {
-    assert nextFreeOffset.get() == UNINITIALIZED;
-    try {
-      if (data == null) {
-        data = ByteBuffer.allocate(this.size);
-      }
-    } catch (OutOfMemoryError e) {
-      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
-      assert failInit; // should be true.
-      throw e;
+  @Override
+  void allocateDataBuffer() {
+    if (data == null) {
+      data = ByteBuffer.allocate(this.size);
+      data.putLong(0, this.getId());
     }
-    // Mark that it's ready for use
-    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
-    // We should always succeed the above CAS since only one thread
-    // calls init()!
-    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 82c2eab..6563122 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -96,6 +96,8 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -2426,6 +2428,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
       final Configuration conf, final HTableDescriptor htd, boolean initialize)
       throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     WAL wal = createWal(conf, rootDir, info);
     return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 422c54b..8d8b6df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -27,6 +27,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -49,8 +50,10 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -397,6 +400,7 @@ public class TestCoprocessorInterface {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
     Path path = new Path(DIR + callingMethod);
     Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index 80d0e3a..b99087d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -47,10 +47,12 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -152,6 +154,7 @@ public class TestRegionObserverScannerOpenHook {
     for (byte[] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     WAL wal = HBaseTestingUtility.createWal(conf, path, info);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
index 2e44dee..15d449d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
@@ -34,7 +34,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -100,6 +102,7 @@ public class TestRegionObserverStacking extends TestCase {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
index f1775d0..fae7247 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
@@ -40,8 +40,10 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -342,6 +344,7 @@ public class TestScannerFromBucketCache {
   private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
       String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
       byte[]... families) throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path logDir = test_util.getDataTestDirOnTestFS(callingMethod + ".log");
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index cc73d9d..32bce26 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -65,6 +65,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActi
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -73,6 +75,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Triple;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -88,6 +91,10 @@ public class TestCatalogJanitor {
   @Rule
   public TestName name = new TestName();
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+  }
   /**
    * Mock MasterServices for tests below.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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 418aadf..096c5ef 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
@@ -241,7 +241,7 @@ public class TestBulkLoad {
     for (byte[] family : families) {
       hTableDescriptor.addFamily(new HColumnDescriptor(family));
     }
-
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     // TODO We need a way to do this without creating files
     return HRegion.createHRegion(hRegionInfo,
         new Path(testFolder.newFolder().toURI()),

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
index 3b4d068..09877b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
@@ -73,7 +73,7 @@ public class TestCellFlatSet extends TestCase {
     descCbOnHeap = new CellArrayMap(CellComparator.COMPARATOR,descCells,0,NUM_OF_CELLS,true);
     CONF.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     CONF.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    MemStoreChunkPool.chunkPoolDisabled = false;
+    ChunkCreator.chunkPoolDisabled = false;
   }
 
   /* Create and test CellSet based on CellArrayMap */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index a888c45..9e90f3e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,7 +51,7 @@ import static org.junit.Assert.assertTrue;
 public class TestCompactingMemStore extends TestDefaultMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingMemStore.class);
-  protected static MemStoreChunkPool chunkPool;
+  protected static ChunkCreator chunkCreator;
   protected HRegion region;
   protected RegionServicesForStores regionServicesForStores;
   protected HStore store;
@@ -65,7 +66,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
 
   @After
   public void tearDown() throws Exception {
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
   }
 
   @Override
@@ -84,15 +85,21 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
     HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
-    this.region = hbaseUtility.createTestRegion("foobar", hcd);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("foobar"));
+    htd.addFamily(hcd);
+    HRegionInfo info =
+        new HRegionInfo(TableName.valueOf("foobar"), null, null, false);
+    WAL wal = hbaseUtility.createWal(conf, hbaseUtility.getDataTestDir(), info);
+    this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf, htd, wal, true);
+    //this.region = hbaseUtility.createTestRegion("foobar", hcd);
     this.regionServicesForStores = region.getRegionServicesForStores();
     this.store = new HStore(region, hcd, conf);
 
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
-        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
-    assertTrue(chunkPool != null);
+    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
+      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+    assertTrue(chunkCreator != null);
   }
 
   /**
@@ -390,7 +397,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -434,16 +441,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
 
@@ -464,7 +471,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   @Test
@@ -516,16 +523,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     memstore.add(new KeyValue(row, fam, qf1, 3, val), null);
     assertEquals(3, memstore.getActive().getCellsCount());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
 
@@ -553,7 +560,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   //////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
index 5a48455..66e107a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
@@ -44,17 +44,13 @@ import java.util.List;
 public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingToCellArrayMapMemStore.class);
-  //private static MemStoreChunkPool chunkPool;
-  //private HRegion region;
-  //private RegionServicesForStores regionServicesForStores;
-  //private HStore store;
 
   //////////////////////////////////////////////////////////////////////////////
   // Helpers
   //////////////////////////////////////////////////////////////////////////////
 
   @Override public void tearDown() throws Exception {
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
   }
 
   @Override public void setUp() throws Exception {
@@ -408,16 +404,16 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkPool.getPoolSize() == 0);
+    assertTrue(chunkCreator.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
 
     // clear chunks
-    chunkPool.clearChunks();
+    chunkCreator.clearChunksInPool();
 
     // Creating another snapshot
 
@@ -438,7 +434,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkPool.getPoolSize() > 0);
+    assertTrue(chunkCreator.getPoolSize() > 0);
   }
 
   @Test
@@ -472,7 +468,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkPool.getPoolSize();
+    int chunkCount = chunkCreator.getPoolSize();
     assertTrue(chunkCount > 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index 8e85730..e320368 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -164,6 +164,7 @@ public class TestCompactionArchiveConcurrentClose {
 
     HRegionFileSystem fs = new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf),
         tableDir, info);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index 89b2368..e7fcf18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -174,6 +174,7 @@ public class TestCompactionArchiveIOException {
   private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
       throws IOException {
     Configuration conf = testUtil.getConfiguration();
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
     Path regionDir = new Path(tableDir, info.getEncodedName());
     Path storeDir = new Path(regionDir, htd.getColumnFamilies()[0].getNameAsString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/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
index 7154511..bff5bec 100644
--- 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
@@ -104,6 +104,7 @@ public class TestCompactionPolicy {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
 
     hlog = new FSHLog(fs, basedir, logName, conf);
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
     region.close();
     Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 7434eb1..41b304b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -84,6 +85,7 @@ public class TestDefaultMemStore {
   protected static final byte[] FAMILY = Bytes.toBytes("column");
   protected MultiVersionConcurrencyControl mvcc;
   protected AtomicLong startSeqNum = new AtomicLong(0);
+  protected ChunkCreator chunkCreator;
 
   private String getName() {
     return this.name.getMethodName();
@@ -92,9 +94,17 @@ public class TestDefaultMemStore {
   @Before
   public void setUp() throws Exception {
     internalSetUp();
+    // no pool
+    this.chunkCreator =
+        ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     this.memstore = new DefaultMemStore();
   }
 
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    ChunkCreator.getInstance().clearChunkIds();
+  }
+
   protected void internalSetUp() throws Exception {
     this.mvcc = new MultiVersionConcurrencyControl();
   }
@@ -129,7 +139,9 @@ public class TestDefaultMemStore {
       assertEquals(Segment.getCellLength(kv), sizeChangeForSecondCell.getDataSize());
       // make sure chunk size increased even when writing the same cell, if using MSLAB
       if (msLab instanceof MemStoreLABImpl) {
-        assertEquals(2 * Segment.getCellLength(kv),
+        // since we add the chunkID at the 0th offset of the chunk and the
+        // chunkid is a long we need to account for those 8 bytes
+        assertEquals(2 * Segment.getCellLength(kv) + Bytes.SIZEOF_LONG,
           ((MemStoreLABImpl) msLab).getCurrentChunk().getNextFreeOffset());
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 73fb9cf..24e850d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -266,6 +266,7 @@ public class TestFailedAppendAndSync {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2c2178b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index b416c7d..0f24a24 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -153,7 +153,7 @@ public class TestHMobStore {
 
     htd.addFamily(hcd);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
     final WALFactory wals = new WALFactory(walConf, null, methodName);


[09/50] [abbrv] hbase git commit: HBASE-14141 HBase Backup/Restore Phase 3: Filter WALs on backup to include only edits from backed up tables (Vladimir Rodionov)

Posted by sy...@apache.org.
HBASE-14141 HBase Backup/Restore Phase 3: Filter WALs on backup to include only edits from backed up tables (Vladimir Rodionov)


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

Branch: refs/heads/hbase-12439
Commit: 910b68082c8f200f0ba6395a76b7ee1c8917e401
Parents: e916b79
Author: tedyu <yu...@gmail.com>
Authored: Tue Apr 4 18:20:11 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Apr 4 18:20:11 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/impl/BackupManager.java |   2 +-
 .../backup/impl/IncrementalBackupManager.java   |  89 ++++++--
 .../impl/IncrementalTableBackupClient.java      | 211 +++++++++++--------
 .../hbase/backup/impl/RestoreTablesClient.java  |   5 +-
 .../hbase/backup/impl/TableBackupClient.java    |   4 -
 .../backup/mapreduce/HFileSplitterJob.java      |   2 +-
 .../backup/mapreduce/MapReduceRestoreJob.java   |  14 +-
 .../hadoop/hbase/backup/util/RestoreTool.java   | 134 ++----------
 .../hadoop/hbase/mapreduce/WALInputFormat.java  | 119 +++++++----
 .../hadoop/hbase/mapreduce/WALPlayer.java       |  10 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java | 101 +++++++++
 11 files changed, 410 insertions(+), 281 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/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
index c09ce48..f09310f 100644
--- 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
@@ -466,7 +466,7 @@ public class BackupManager implements Closeable {
 
   /**
    * 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
+   * TTL expiration and are used by Backup Log Cleaner plug-in to determine which WAL files can be
    * safely purged.
    */
   public void recordWALFiles(List<String> files) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/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
index 0f1453e..6330899 100644
--- 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
@@ -21,8 +21,10 @@ package org.apache.hadoop.hbase.backup.impl;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,7 +35,6 @@ 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.BackupInfo;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable.WALItem;
 import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
@@ -59,12 +60,10 @@ public class IncrementalBackupManager extends BackupManager {
   /**
    * Obtain the list of logs that need to be copied out for this incremental backup. The list is set
    * in BackupInfo.
-   * @param conn the Connection
-   * @param backupInfo backup info
-   * @return The new HashMap of RS log timestamps after the log roll for this incremental backup.
+   * @return The new HashMap of RS log time stamps after the log roll for this incremental backup.
    * @throws IOException exception
    */
-  public HashMap<String, Long> getIncrBackupLogFileList(Connection conn, BackupInfo backupInfo)
+  public HashMap<String, Long> getIncrBackupLogFileMap()
       throws IOException {
     List<String> logList;
     HashMap<String, Long> newTimestamps;
@@ -105,40 +104,84 @@ public class IncrementalBackupManager extends BackupManager {
     List<WALItem> logFromSystemTable =
         getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo()
             .getBackupRootDir());
-    addLogsFromBackupSystemToContext(logFromSystemTable);
-
     logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
     backupInfo.setIncrBackupFileList(logList);
 
     return newTimestamps;
   }
 
-  private List<String> excludeAlreadyBackedUpWALs(List<String> logList,
-      List<WALItem> logFromSystemTable) {
+  /**
+   * Get list of WAL files eligible for incremental backup
+   * @return list of WAL files
+   * @throws IOException
+   */
+  public List<String> getIncrBackupLogFileList()
+      throws IOException {
+    List<String> logList;
+    HashMap<String, Long> newTimestamps;
+    HashMap<String, Long> previousTimestampMins;
+
+    String savedStartCode = readBackupStartCode();
+
+    // key: tableName
+    // value: <RegionServer,PreviousTimeStamp>
+    HashMap<TableName, HashMap<String, Long>> previousTimestampMap = readLogTimestampMap();
+
+    previousTimestampMins = BackupUtils.getRSLogTimestampMins(previousTimestampMap);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("StartCode " + savedStartCode + "for backupID " + backupInfo.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 backup system table. "
+              + "In order to create an incremental backup, at least one full backup is needed.");
+    }
+
+    newTimestamps = readRegionServerLastLogRollResult();
+
+    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
+    List<WALItem> logFromSystemTable =
+        getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo()
+            .getBackupRootDir());
+
+    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
+    backupInfo.setIncrBackupFileList(logList);
 
-    List<String> backupedWALList = toWALList(logFromSystemTable);
-    logList.removeAll(backupedWALList);
     return logList;
   }
 
-  private List<String> toWALList(List<WALItem> logFromSystemTable) {
 
-    List<String> list = new ArrayList<String>(logFromSystemTable.size());
-    for (WALItem item : logFromSystemTable) {
-      list.add(item.getWalFile());
+  private List<String> excludeAlreadyBackedUpWALs(List<String> logList,
+      List<WALItem> logFromSystemTable) {
+
+    Set<String> walFileNameSet = convertToSet(logFromSystemTable);
+
+    List<String> list = new ArrayList<String>();
+    for (int i=0; i < logList.size(); i++) {
+      Path p = new Path(logList.get(i));
+      String name  = p.getName();
+      if (walFileNameSet.contains(name)) continue;
+      list.add(logList.get(i));
     }
     return list;
   }
 
-  private void addLogsFromBackupSystemToContext(List<WALItem> logFromSystemTable) {
-    List<String> walFiles = new ArrayList<String>();
-    for (WALItem item : logFromSystemTable) {
-      Path p = new Path(item.getWalFile());
-      String walFileName = p.getName();
-      String backupId = item.getBackupId();
-      String relWALPath = backupId + Path.SEPARATOR + walFileName;
-      walFiles.add(relWALPath);
+  /**
+   * Create Set of WAL file names (not full path names)
+   * @param logFromSystemTable
+   * @return set of WAL file names
+   */
+  private Set<String> convertToSet(List<WALItem> logFromSystemTable) {
+
+    Set<String> set = new HashSet<String>();
+    for (int i=0; i < logFromSystemTable.size(); i++) {
+      WALItem item = logFromSystemTable.get(i);
+      set.add(item.walFile);
     }
+    return set;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
index 8f6f264..3003c93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.commons.lang.StringUtils;
@@ -34,7 +35,6 @@ import org.apache.commons.logging.LogFactory;
 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.TableName;
 import org.apache.hadoop.hbase.backup.BackupCopyJob;
 import org.apache.hadoop.hbase.backup.BackupInfo;
@@ -45,11 +45,15 @@ import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
 import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.mapreduce.WALPlayer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.util.Tool;
 
 /**
  * Incremental backup implementation.
@@ -69,7 +73,8 @@ public class IncrementalTableBackupClient extends TableBackupClient {
     FileSystem fs = FileSystem.get(conf);
     List<String> list = new ArrayList<String>();
     for (String file : incrBackupFileList) {
-      if (fs.exists(new Path(file))) {
+      Path p = new Path(file);
+      if (fs.exists(p) || isActiveWalPath(p)) {
         list.add(file);
       } else {
         LOG.warn("Can't find file: " + file);
@@ -78,90 +83,13 @@ public class IncrementalTableBackupClient extends TableBackupClient {
     return list;
   }
 
-  private List<String> getMissingFiles(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);
-      }
-    }
-    return list;
-
-  }
-
   /**
-   * Do incremental copy.
-   * @param backupInfo backup info
+   * Check if a given path is belongs to active WAL directory
+   * @param p path
+   * @return true, if yes
    */
-  private void incrementalCopy(BackupInfo backupInfo) throws Exception {
-
-    LOG.info("Incremental copy is starting.");
-    // set overall backup phase: incremental_copy
-    backupInfo.setPhase(BackupPhase.INCREMENTAL_COPY);
-    // get incremental backup file list and prepare parms for DistCp
-    List<String> incrBackupFileList = backupInfo.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] = backupInfo.getHLogTargetDir();
-
-    BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
-    int counter = 0;
-    int MAX_ITERAIONS = 2;
-    while (counter++ < MAX_ITERAIONS) {
-      // We run DistCp maximum 2 times
-      // If it fails on a second time, we throw Exception
-      int res =
-          copyService.copy(backupInfo, backupManager, conf, BackupType.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 "
-            + StringUtils.join(incrBackupFileList, ",") + " to "
-            + backupInfo.getHLogTargetDir());
-      }
-      List<String> missingFiles = getMissingFiles(incrBackupFileList);
-
-      if (missingFiles.isEmpty()) {
-        break;
-      } else {
-        // Repeat DistCp, some files have been moved from WALs to oldWALs during previous run
-        // update backupInfo and strAttr
-        if (counter == MAX_ITERAIONS) {
-          String msg =
-              "DistCp could not finish the following files: " + StringUtils.join(missingFiles, ",");
-          LOG.error(msg);
-          throw new IOException(msg);
-        }
-        List<String> converted = convertFilesFromWALtoOldWAL(missingFiles);
-        incrBackupFileList.removeAll(missingFiles);
-        incrBackupFileList.addAll(converted);
-        backupInfo.setIncrBackupFileList(incrBackupFileList);
-
-        // Run DistCp only for missing files (which have been moved from WALs to oldWALs
-        // during previous run)
-        strArr = converted.toArray(new String[converted.size() + 1]);
-        strArr[strArr.length - 1] = backupInfo.getHLogTargetDir();
-      }
-    }
-
-    LOG.info("Incremental copy from " + StringUtils.join(incrBackupFileList, ",") + " to "
-        + backupInfo.getHLogTargetDir() + " finished.");
-  }
-
-  private List<String> convertFilesFromWALtoOldWAL(List<String> missingFiles) throws IOException {
-    List<String> list = new ArrayList<String>();
-    for (String path : missingFiles) {
-      if (path.indexOf(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME) < 0) {
-        LOG.error("Copy incremental log files failed, file is missing : " + path);
-        throw new IOException("Failed of Hadoop Distributed Copy to "
-            + backupInfo.getHLogTargetDir() + ", file is missing " + path);
-      }
-      list.add(path.replace(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME, Path.SEPARATOR
-          + HConstants.HREGION_OLDLOGDIR_NAME));
-    }
-    return list;
+  private boolean isActiveWalPath(Path p) {
+    return !AbstractFSWALProvider.isArchivedLogFile(p);
   }
 
   static int getIndex(TableName tbl, List<TableName> sTableList) {
@@ -286,7 +214,7 @@ public class IncrementalTableBackupClient extends TableBackupClient {
         + backupManager.getIncrementalBackupTableSet());
     try {
       newTimestamps =
-          ((IncrementalBackupManager) backupManager).getIncrBackupLogFileList(conn, backupInfo);
+          ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
     } catch (Exception e) {
       // fail the overall backup and return
       failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
@@ -297,13 +225,16 @@ public class IncrementalTableBackupClient extends TableBackupClient {
     try {
       // copy out the table and region info files for each table
       BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
-      incrementalCopy(backupInfo);
+      // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
+      convertWALsToHFiles(backupInfo);
+      incrementalCopyHFiles(backupInfo);
       // Save list of WAL files copied
       backupManager.recordWALFiles(backupInfo.getIncrBackupFileList());
     } catch (Exception e) {
       String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId;
       // fail the overall backup and return
       failBackup(conn, backupInfo, backupManager, e, msg, BackupType.INCREMENTAL, conf);
+      return;
     }
     // case INCR_BACKUP_COMPLETE:
     // set overall backup status: complete. Here we make sure to complete the backup.
@@ -323,8 +254,7 @@ public class IncrementalTableBackupClient extends TableBackupClient {
           backupManager.readLogTimestampMap();
 
       Long newStartCode =
-          BackupUtils.getMinValue(BackupUtils
-              .getRSLogTimestampMins(newTableSetTimestampMap));
+          BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap));
       backupManager.writeBackupStartCode(newStartCode);
 
       handleBulkLoad(backupInfo.getTableNames());
@@ -337,4 +267,109 @@ public class IncrementalTableBackupClient extends TableBackupClient {
     }
   }
 
+  private void incrementalCopyHFiles(BackupInfo backupInfo) throws Exception {
+
+    try {
+      LOG.debug("Incremental copy HFiles is starting.");
+      // set overall backup phase: incremental_copy
+      backupInfo.setPhase(BackupPhase.INCREMENTAL_COPY);
+      // get incremental backup file list and prepare parms for DistCp
+      List<String> incrBackupFileList = new ArrayList<String>();
+      // Add Bulk output
+      incrBackupFileList.add(getBulkOutputDir().toString());
+      String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+      strArr[strArr.length - 1] = backupInfo.getBackupRootDir();
+      BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
+      int res = copyService.copy(backupInfo, backupManager, conf, BackupType.INCREMENTAL, strArr);
+      if (res != 0) {
+        LOG.error("Copy incremental HFile files failed with return code: " + res + ".");
+        throw new IOException("Failed copy from " + StringUtils.join(incrBackupFileList, ',')
+            + " to " + backupInfo.getHLogTargetDir());
+      }
+      LOG.debug("Incremental copy HFiles from " + StringUtils.join(incrBackupFileList, ',')
+          + " to " + backupInfo.getBackupRootDir() + " finished.");
+    } finally {
+      deleteBulkLoadDirectory();
+    }
+  }
+
+  private void deleteBulkLoadDirectory() throws IOException {
+    // delete original bulk load directory on method exit
+    Path path = getBulkOutputDir();
+    FileSystem fs = FileSystem.get(conf);
+    boolean result = fs.delete(path, true);
+    if (!result) {
+      LOG.warn("Could not delete " + path);
+    }
+
+  }
+
+  private void convertWALsToHFiles(BackupInfo backupInfo) throws IOException {
+    // get incremental backup file list and prepare parameters for DistCp
+    List<String> incrBackupFileList = backupInfo.getIncrBackupFileList();
+    // Get list of tables in incremental backup set
+    Set<TableName> tableSet = backupManager.getIncrementalBackupTableSet();
+    // filter missing files out (they have been copied by previous backups)
+    incrBackupFileList = filterMissingFiles(incrBackupFileList);
+    for (TableName table : tableSet) {
+      // Check if table exists
+      if (tableExists(table, conn)) {
+        walToHFiles(incrBackupFileList, table);
+      } else {
+        LOG.warn("Table " + table + " does not exists. Skipping in WAL converter");
+      }
+    }
+  }
+
+
+  private boolean tableExists(TableName table, Connection conn) throws IOException {
+    try (Admin admin = conn.getAdmin();) {
+      return admin.tableExists(table);
+    }
+  }
+
+  private void walToHFiles(List<String> dirPaths, TableName tableName) throws IOException {
+
+    Tool player = new WALPlayer();
+
+    // Player reads all files in arbitrary directory structure and creates
+    // a Map task for each file. We use ';' as separator
+    // because WAL file names contains ','
+    String dirs = StringUtils.join(dirPaths, ';');
+
+    Path bulkOutputPath = getBulkOutputDirForTable(tableName);
+    conf.set(WALPlayer.BULK_OUTPUT_CONF_KEY, bulkOutputPath.toString());
+    conf.set(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ";");
+    String[] playerArgs = { dirs, tableName.getNameAsString() };
+
+    try {
+      player.setConf(conf);
+      int result = player.run(playerArgs);
+      if(result != 0) {
+        throw new IOException("WAL Player failed");
+      }
+      conf.unset(WALPlayer.INPUT_FILES_SEPARATOR_KEY);
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception ee) {
+      throw new IOException("Can not convert from directory " + dirs
+          + " (check Hadoop, HBase and WALPlayer M/R job logs) ", ee);
+    }
+  }
+
+  private Path getBulkOutputDirForTable(TableName table) {
+    Path tablePath = getBulkOutputDir();
+    tablePath = new Path(tablePath, table.getNamespaceAsString());
+    tablePath = new Path(tablePath, table.getQualifierAsString());
+    return new Path(tablePath, "data");
+  }
+
+  private Path getBulkOutputDir() {
+    String backupId = backupInfo.getBackupId();
+    Path path = new Path(backupInfo.getBackupRootDir());
+    path = new Path(path, ".tmp");
+    path = new Path(path, backupId);
+    return path;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
index 2e4ecce..381e9b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
@@ -169,8 +169,9 @@ public class RestoreTablesClient {
     // full backup path comes first
     for (int i = 1; i < images.length; i++) {
       BackupImage im = images[i];
-      String logBackupDir = HBackupFileSystem.getLogBackupDir(im.getRootDir(), im.getBackupId());
-      dirList.add(new Path(logBackupDir));
+      String fileBackupDir = HBackupFileSystem.getTableBackupDir(im.getRootDir(),
+                  im.getBackupId(), sTable)+ Path.SEPARATOR+"data";
+      dirList.add(new Path(fileBackupDir));
     }
 
     String dirs = StringUtils.join(dirList, ",");

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
index 42a8076..125b5da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
@@ -351,10 +351,6 @@ public abstract class TableBackupClient {
     // add and store the manifest for the backup
     addManifest(backupInfo, backupManager, type, conf);
 
-    // 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 =
         obtainBackupMetaDataStr(backupInfo) + ",startts=" + backupInfo.getStartTs()

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
index 5641720..604e502 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
@@ -107,11 +107,11 @@ public class HFileSplitterJob extends Configured implements Tool {
     String inputDirs = args[0];
     String tabName = args[1];
     conf.setStrings(TABLES_KEY, tabName);
+    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
     Job job =
         Job.getInstance(conf,
           conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
     job.setJarByClass(HFileSplitterJob.class);
-    FileInputFormat.addInputPaths(job, inputDirs);
     job.setInputFormatClass(HFileInputFormat.class);
     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
index 9bafe12..4161ca9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
@@ -62,13 +62,8 @@ public class MapReduceRestoreJob implements RestoreJob {
 
     String bulkOutputConfKey;
 
-    if (fullBackupRestore) {
-      player = new HFileSplitterJob();
-      bulkOutputConfKey = HFileSplitterJob.BULK_OUTPUT_CONF_KEY;
-    } else {
-      player = new WALPlayer();
-      bulkOutputConfKey = WALPlayer.BULK_OUTPUT_CONF_KEY;
-    }
+    player = new HFileSplitterJob();
+    bulkOutputConfKey = HFileSplitterJob.BULK_OUTPUT_CONF_KEY;
     // Player reads all files in arbitrary directory structure and creates
     // a Map task for each file
     String dirs = StringUtils.join(dirPaths, ",");
@@ -88,7 +83,10 @@ public class MapReduceRestoreJob implements RestoreJob {
       Path bulkOutputPath = getBulkOutputDir(getFileNameCompatibleString(newTableNames[i]));
       Configuration conf = getConf();
       conf.set(bulkOutputConfKey, bulkOutputPath.toString());
-      String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+      String[] playerArgs =
+        { dirs,
+          fullBackupRestore? newTableNames[i].getNameAsString():tableNames[i].getNameAsString()
+        };
 
       int result = 0;
       int loaderResult = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index 79adcab..d34701f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -46,8 +46,6 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-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.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
@@ -63,19 +61,13 @@ import org.apache.hadoop.hbase.util.FSTableDescriptors;
 public class RestoreTool {
 
   public static final Log LOG = LogFactory.getLog(BackupUtils.class);
-
-  private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR };
-
   private final static long TABLE_AVAILABILITY_WAIT_TIME = 180000;
 
+  private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR };
   protected Configuration conf = null;
-
   protected Path backupRootPath;
-
   protected String backupId;
-
   protected FileSystem fs;
-  private final Path restoreTmpPath;
 
   // store table name and snapshot dir mapping
   private final HashMap<TableName, Path> snapshotMap = new HashMap<>();
@@ -86,9 +78,6 @@ public class RestoreTool {
     this.backupRootPath = backupRootPath;
     this.backupId = backupId;
     this.fs = backupRootPath.getFileSystem(conf);
-    this.restoreTmpPath =
-        new Path(conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
-          HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY), "restore");
   }
 
   /**
@@ -218,7 +207,7 @@ public class RestoreTool {
   public void fullRestoreTable(Connection conn, Path tableBackupPath, TableName tableName,
       TableName newTableName, boolean truncateIfExists, String lastIncrBackupId)
           throws IOException {
-    restoreTableAndCreate(conn, tableName, newTableName, tableBackupPath, truncateIfExists,
+    createAndRestoreTable(conn, tableName, newTableName, tableBackupPath, truncateIfExists,
       lastIncrBackupId);
   }
 
@@ -281,48 +270,6 @@ public class RestoreTool {
     return tableDescriptor;
   }
 
-  /**
-   * Duplicate the backup image if it's on local cluster
-   * @see HStore#bulkLoadHFile(org.apache.hadoop.hbase.regionserver.StoreFile)
-   * @see HRegionFileSystem#bulkLoadStoreFile(String familyName, Path srcPath, long seqNum)
-   * @param tableArchivePath archive path
-   * @return the new tableArchivePath
-   * @throws IOException exception
-   */
-  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;
-  }
-
   private HTableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName,
       String lastIncrBackupId) throws IOException {
     if (lastIncrBackupId != null) {
@@ -334,7 +281,7 @@ public class RestoreTool {
     return null;
   }
 
-  private void restoreTableAndCreate(Connection conn, TableName tableName, TableName newTableName,
+  private void createAndRestoreTable(Connection conn, TableName tableName, TableName newTableName,
       Path tableBackupPath, boolean truncateIfExists, String lastIncrBackupId) throws IOException {
     if (newTableName == null) {
       newTableName = tableName;
@@ -403,33 +350,13 @@ public class RestoreTool {
       // the regions in fine grain
       checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, regionPathList,
         tableDescriptor, truncateIfExists);
-      if (tableArchivePath != null) {
-        // start real restore through bulkload
-        // if the backup target is on local cluster, special action needed
-        Path tempTableArchivePath = checkLocalAndBackup(tableArchivePath);
-        if (tempTableArchivePath.equals(tableArchivePath)) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath);
-          }
-        } else {
-          regionPathList = getRegionList(tempTableArchivePath); // point to the tempDir
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath);
-          }
-        }
+      RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
+      Path[] paths = new Path[regionPathList.size()];
+      regionPathList.toArray(paths);
+      restoreService.run(paths, new TableName[]{tableName}, new TableName[] {newTableName}, true);
 
-        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) {
+      LOG.error(e);
       throw new IllegalStateException("Cannot restore hbase table", e);
     }
   }
@@ -453,28 +380,6 @@ public class RestoreTool {
   }
 
   /**
-   * 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 {
-
-    // By default, it is 32 and loader will fail if # of files in any region exceed this
-    // limit. Bad for snapshot restore.
-    this.conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
-    this.conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
-    LoadIncrementalHFiles loader = null;
-    try {
-      loader = new LoadIncrementalHFiles(this.conf);
-    } catch (Exception e1) {
-      throw new IOException(e1);
-    }
-    return loader;
-  }
-
-  /**
    * 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
@@ -591,17 +496,18 @@ public class RestoreTool {
           // create table using table descriptor and region boundaries
           admin.createTable(htd, keys);
         }
-        long startTime = EnvironmentEdgeManager.currentTime();
-        while (!admin.isTableAvailable(targetTableName, keys)) {
-          try {
-            Thread.sleep(100);
-          } catch (InterruptedException ie) {
-            Thread.currentThread().interrupt();
-          }
-          if (EnvironmentEdgeManager.currentTime() - startTime > TABLE_AVAILABILITY_WAIT_TIME) {
-            throw new IOException("Time out " + TABLE_AVAILABILITY_WAIT_TIME + "ms expired, table "
-                + targetTableName + " is still not available");
-          }
+
+      }
+      long startTime = EnvironmentEdgeManager.currentTime();
+      while (!admin.isTableAvailable(targetTableName)) {
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+        }
+        if (EnvironmentEdgeManager.currentTime() - startTime > TABLE_AVAILABILITY_WAIT_TIME) {
+          throw new IOException("Time out " + TABLE_AVAILABILITY_WAIT_TIME + "ms expired, table "
+              + targetTableName + " is still not available");
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
index 0ca78b4..8b4e967 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.mapreduce;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.EOFException;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -35,10 +36,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
-import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputFormat;
@@ -46,6 +47,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -142,56 +144,89 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
     Entry currentEntry = new Entry();
     private long startTime;
     private long endTime;
+    private Configuration conf;
+    private Path logFile;
+    private long currentPos;
 
     @Override
     public void initialize(InputSplit split, TaskAttemptContext context)
         throws IOException, InterruptedException {
       WALSplit hsplit = (WALSplit)split;
-      Path logFile = new Path(hsplit.getLogFileName());
-      Configuration conf = context.getConfiguration();
+      logFile = new Path(hsplit.getLogFileName());
+      conf = context.getConfiguration();
       LOG.info("Opening reader for "+split);
-      try {
-        this.reader = WALFactory.createReader(logFile.getFileSystem(conf), logFile, conf);
-      } catch (EOFException x) {
-        LOG.info("Ignoring corrupted WAL file: " + logFile
-            + " (This is normal when a RegionServer crashed.)");
-        this.reader = null;
-      }
+      openReader(logFile);
       this.startTime = hsplit.getStartTime();
       this.endTime = hsplit.getEndTime();
     }
 
+    private void openReader(Path path) throws IOException
+    {
+      closeReader();
+      reader = AbstractFSWALProvider.openReader(path, conf);
+      seek();
+      setCurrentPath(path);
+    }
+
+    private void setCurrentPath(Path path) {
+      this.logFile = path;
+    }
+
+    private void closeReader() throws IOException {
+      if (reader != null) {
+        reader.close();
+        reader = null;
+      }
+    }
+
+    private void seek() throws IOException {
+      if (currentPos != 0) {
+        reader.seek(currentPos);
+      }
+    }
+
     @Override
     public boolean nextKeyValue() throws IOException, InterruptedException {
       if (reader == null) return false;
-
+      this.currentPos = reader.getPosition();
       Entry temp;
       long i = -1;
-      do {
-        // skip older entries
-        try {
-          temp = reader.next(currentEntry);
-          i++;
-        } catch (EOFException x) {
-          LOG.warn("Corrupted entry detected. Ignoring the rest of the file."
-              + " (This is normal when a RegionServer crashed.)");
+      try {
+        do {
+          // skip older entries
+          try {
+            temp = reader.next(currentEntry);
+            i++;
+          } catch (EOFException x) {
+            LOG.warn("Corrupted entry detected. Ignoring the rest of the file."
+                + " (This is normal when a RegionServer crashed.)");
+            return false;
+          }
+        } while (temp != null && temp.getKey().getWriteTime() < startTime);
+
+        if (temp == null) {
+          if (i > 0) LOG.info("Skipped " + i + " entries.");
+          LOG.info("Reached end of file.");
           return false;
+        } else if (i > 0) {
+          LOG.info("Skipped " + i + " entries, until ts: " + temp.getKey().getWriteTime() + ".");
+        }
+        boolean res = temp.getKey().getWriteTime() <= endTime;
+        if (!res) {
+          LOG.info("Reached ts: " + temp.getKey().getWriteTime()
+              + " ignoring the rest of the file.");
+        }
+        return res;
+      } catch (IOException e) {
+        Path archivedLog = AbstractFSWALProvider.getArchivedLogPath(logFile, conf);
+        if (logFile != archivedLog) {
+          openReader(archivedLog);
+          // Try call again in recursion
+          return nextKeyValue();
+        } else {
+          throw e;
         }
       }
-      while(temp != null && temp.getKey().getWriteTime() < startTime);
-
-      if (temp == null) {
-        if (i > 0) LOG.info("Skipped " + i + " entries.");
-        LOG.info("Reached end of file.");
-        return false;
-      } else if (i > 0) {
-        LOG.info("Skipped " + i + " entries, until ts: " + temp.getKey().getWriteTime() + ".");
-      }
-      boolean res = temp.getKey().getWriteTime() <= endTime;
-      if (!res) {
-        LOG.info("Reached ts: " + temp.getKey().getWriteTime() + " ignoring the rest of the file.");
-      }
-      return res;
     }
 
     @Override
@@ -235,6 +270,7 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
   List<InputSplit> getSplits(final JobContext context, final String startKey, final String endKey)
       throws IOException, InterruptedException {
     Configuration conf = context.getConfiguration();
+    boolean ignoreMissing = conf.getBoolean(WALPlayer.IGNORE_MISSING_FILES, false);
     Path[] inputPaths = getInputPaths(conf);
     long startTime = conf.getLong(startKey, Long.MIN_VALUE);
     long endTime = conf.getLong(endKey, Long.MAX_VALUE);
@@ -242,8 +278,16 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
     List<FileStatus> allFiles = new ArrayList<FileStatus>();
     for(Path inputPath: inputPaths){
       FileSystem fs = inputPath.getFileSystem(conf);
-      List<FileStatus> files = getFiles(fs, inputPath, startTime, endTime);
-      allFiles.addAll(files);
+      try {
+        List<FileStatus> files = getFiles(fs, inputPath, startTime, endTime);
+        allFiles.addAll(files);
+      } catch (FileNotFoundException e) {
+        if (ignoreMissing) {
+          LOG.warn("File "+ inputPath +" is missing. Skipping it.");
+          continue;
+        }
+        throw e;
+      }
     }
     List<InputSplit> splits = new ArrayList<InputSplit>(allFiles.size());
     for (FileStatus file : allFiles) {
@@ -253,8 +297,9 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
   }
 
   private Path[] getInputPaths(Configuration conf) {
-    String inpDirs = conf.get("mapreduce.input.fileinputformat.inputdir");
-    return StringUtils.stringToPath(inpDirs.split(","));
+    String inpDirs = conf.get(FileInputFormat.INPUT_DIR);
+    return StringUtils.stringToPath(
+      inpDirs.split(conf.get(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ",")));
   }
 
   private List<FileStatus> getFiles(FileSystem fs, Path dir, long startTime, long endTime)

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/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 d16dcf5..d15ffcf 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
@@ -58,7 +58,7 @@ import org.apache.hadoop.util.ToolRunner;
 /**
  * A tool to replay WAL files as a M/R job.
  * The WAL can be replayed for a set of tables or all tables,
- * and a timerange can be provided (in milliseconds).
+ * and a time range can be provided (in milliseconds).
  * The WAL is filtered to the passed set of tables and  the output
  * can optionally be mapped to another set of tables.
  *
@@ -73,6 +73,9 @@ public class WALPlayer extends Configured implements Tool {
   public final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
   public final static String TABLES_KEY = "wal.input.tables";
   public final static String TABLE_MAP_KEY = "wal.input.tablesmap";
+  public final static String INPUT_FILES_SEPARATOR_KEY = "wal.input.separator";
+  public final static String IGNORE_MISSING_FILES = "wal.input.ignore.missing.files";
+
 
   // This relies on Hadoop Configuration to handle warning about deprecated configs and
   // to set the correct non-deprecated configs when an old one shows up.
@@ -128,7 +131,9 @@ public class WALPlayer extends Configured implements Tool {
         throw new IOException("Exactly one table must be specified for bulk HFile case.");
       }
       table = Bytes.toBytes(tables[0]);
+
     }
+
   }
 
   /**
@@ -280,11 +285,10 @@ public class WALPlayer extends Configured implements Tool {
     }
     conf.setStrings(TABLES_KEY, tables);
     conf.setStrings(TABLE_MAP_KEY, tableMap);
+    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
     Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + System.currentTimeMillis()));
     job.setJarByClass(WALPlayer.class);
 
-    FileInputFormat.addInputPaths(job, inputDirs);
-
     job.setInputFormatClass(WALInputFormat.class);
     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/910b6808/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index bf14933..28b7fda 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.wal;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -36,6 +37,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -374,6 +378,103 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   }
 
   /**
+   * Get the archived WAL file path
+   * @param path - active WAL file path
+   * @param conf - configuration
+   * @return archived path if exists, path - otherwise
+   * @throws IOException exception
+   */
+  public static Path getArchivedLogPath(Path path, Configuration conf) throws IOException {
+    Path rootDir = FSUtils.getRootDir(conf);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    Path archivedLogLocation = new Path(oldLogDir, path.getName());
+    final FileSystem fs = FSUtils.getCurrentFileSystem(conf);
+
+    if (fs.exists(archivedLogLocation)) {
+      LOG.info("Log " + path + " was moved to " + archivedLogLocation);
+      return archivedLogLocation;
+    } else {
+      LOG.error("Couldn't locate log: " + path);
+      return path;
+    }
+  }
+
+  /**
+   * Opens WAL reader with retries and
+   * additional exception handling
+   * @param path path to WAL file
+   * @param conf configuration
+   * @return WAL Reader instance
+   * @throws IOException
+   */
+  public static org.apache.hadoop.hbase.wal.WAL.Reader
+    openReader(Path path, Configuration conf)
+        throws IOException
+
+  {
+    long retryInterval = 2000; // 2 sec
+    int maxAttempts = 30;
+    int attempt = 0;
+    Exception ee = null;
+    org.apache.hadoop.hbase.wal.WAL.Reader reader = null;
+    while (reader == null && attempt++ < maxAttempts) {
+      try {
+        // Detect if this is a new file, if so get a new reader else
+        // reset the current reader so that we see the new data
+        reader = WALFactory.createReader(path.getFileSystem(conf), path, conf);
+        return reader;
+      } catch (FileNotFoundException fnfe) {
+        // If the log was archived, continue reading from there
+        Path archivedLog = AbstractFSWALProvider.getArchivedLogPath(path, conf);
+        if (path != archivedLog) {
+          return openReader(archivedLog, conf);
+        } else {
+          throw fnfe;
+        }
+      } catch (LeaseNotRecoveredException lnre) {
+        // HBASE-15019 the WAL was not closed due to some hiccup.
+        LOG.warn("Try to recover the WAL lease " + path, lnre);
+        recoverLease(conf, path);
+        reader = null;
+        ee = lnre;
+      } catch (NullPointerException npe) {
+        // Workaround for race condition in HDFS-4380
+        // which throws a NPE if we open a file before any data node has the most recent block
+        // Just sleep and retry. Will require re-reading compressed WALs for compressionContext.
+        LOG.warn("Got NPE opening reader, will retry.");
+        reader = null;
+        ee = npe;
+      }
+      if (reader == null) {
+        // sleep before next attempt
+        try {
+          Thread.sleep(retryInterval);
+        } catch (InterruptedException e) {
+        }
+      }
+    }
+    throw new IOException("Could not open reader", ee);
+  }
+
+  // For HBASE-15019
+  private static void recoverLease(final Configuration conf, final Path path) {
+    try {
+      final FileSystem dfs = FSUtils.getCurrentFileSystem(conf);
+      FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
+      fsUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
+        @Override
+        public boolean progress() {
+          LOG.debug("Still trying to recover WAL lease: " + path);
+          return true;
+        }
+      });
+    } catch (IOException e) {
+      LOG.warn("unable to recover lease for WAL: " + path, e);
+    }
+  }
+
+
+  /**
    * Get prefix of the log from its name, assuming WAL name in format of
    * log_prefix.filenumber.log_suffix
    * @param name Name of the WAL to parse


[07/50] [abbrv] hbase git commit: HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DescriptorProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DescriptorProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DescriptorProtos.java
index 99dfec2..0468e6c 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DescriptorProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DescriptorProtos.java
@@ -223,7 +223,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getFileCount() > 0) {
         hash = (37 * hash) + FILE_FIELD_NUMBER;
         hash = (53 * hash) + getFileList().hashCode();
@@ -2062,7 +2062,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -5283,7 +5283,7 @@ public final class DescriptorProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasStart()) {
           hash = (37 * hash) + START_FIELD_NUMBER;
           hash = (53 * hash) + getStart();
@@ -5874,7 +5874,7 @@ public final class DescriptorProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasStart()) {
           hash = (37 * hash) + START_FIELD_NUMBER;
           hash = (53 * hash) + getStart();
@@ -6803,7 +6803,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -9930,6 +9930,9 @@ public final class DescriptorProtos {
       /**
        * <pre>
        * Tag-delimited aggregate.
+       * Group type is deprecated and not supported in proto3. However, Proto3
+       * implementations should still be able to parse the group wire format and
+       * treat group fields as unknown fields.
        * </pre>
        *
        * <code>TYPE_GROUP = 10;</code>
@@ -10039,6 +10042,9 @@ public final class DescriptorProtos {
       /**
        * <pre>
        * Tag-delimited aggregate.
+       * Group type is deprecated and not supported in proto3. However, Proto3
+       * implementations should still be able to parse the group wire format and
+       * treat group fields as unknown fields.
        * </pre>
        *
        * <code>TYPE_GROUP = 10;</code>
@@ -10193,10 +10199,6 @@ public final class DescriptorProtos {
        */
       LABEL_REQUIRED(2),
       /**
-       * <pre>
-       * TODO(sanjay): Should we add LABEL_MAP?
-       * </pre>
-       *
        * <code>LABEL_REPEATED = 3;</code>
        */
       LABEL_REPEATED(3),
@@ -10215,10 +10217,6 @@ public final class DescriptorProtos {
        */
       public static final int LABEL_REQUIRED_VALUE = 2;
       /**
-       * <pre>
-       * TODO(sanjay): Should we add LABEL_MAP?
-       * </pre>
-       *
        * <code>LABEL_REPEATED = 3;</code>
        */
       public static final int LABEL_REPEATED_VALUE = 3;
@@ -10854,7 +10852,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -12376,7 +12374,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -13225,7 +13223,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -14320,7 +14318,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -15214,7 +15212,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -16561,7 +16559,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -17744,6 +17742,41 @@ public final class DescriptorProtos {
 
     /**
      * <pre>
+     * By default Swift generators will take the proto package and CamelCase it
+     * replacing '.' with underscore and use that to prefix the types/symbols
+     * defined. When this options is provided, they will use this value instead
+     * to prefix the types/symbols defined.
+     * </pre>
+     *
+     * <code>optional string swift_prefix = 39;</code>
+     */
+    boolean hasSwiftPrefix();
+    /**
+     * <pre>
+     * By default Swift generators will take the proto package and CamelCase it
+     * replacing '.' with underscore and use that to prefix the types/symbols
+     * defined. When this options is provided, they will use this value instead
+     * to prefix the types/symbols defined.
+     * </pre>
+     *
+     * <code>optional string swift_prefix = 39;</code>
+     */
+    java.lang.String getSwiftPrefix();
+    /**
+     * <pre>
+     * By default Swift generators will take the proto package and CamelCase it
+     * replacing '.' with underscore and use that to prefix the types/symbols
+     * defined. When this options is provided, they will use this value instead
+     * to prefix the types/symbols defined.
+     * </pre>
+     *
+     * <code>optional string swift_prefix = 39;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getSwiftPrefixBytes();
+
+    /**
+     * <pre>
      * The parser stores options it doesn't recognize here. See above.
      * </pre>
      *
@@ -17813,6 +17846,7 @@ public final class DescriptorProtos {
       ccEnableArenas_ = false;
       objcClassPrefix_ = "";
       csharpNamespace_ = "";
+      swiftPrefix_ = "";
       uninterpretedOption_ = java.util.Collections.emptyList();
     }
 
@@ -17925,10 +17959,16 @@ public final class DescriptorProtos {
               csharpNamespace_ = bs;
               break;
             }
+            case 314: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00004000;
+              swiftPrefix_ = bs;
+              break;
+            }
             case 7994: {
-              if (!((mutable_bitField0_ & 0x00004000) == 0x00004000)) {
+              if (!((mutable_bitField0_ & 0x00008000) == 0x00008000)) {
                 uninterpretedOption_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption>();
-                mutable_bitField0_ |= 0x00004000;
+                mutable_bitField0_ |= 0x00008000;
               }
               uninterpretedOption_.add(
                   input.readMessage(org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption.PARSER, extensionRegistry));
@@ -17942,7 +17982,7 @@ public final class DescriptorProtos {
         throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00004000) == 0x00004000)) {
+        if (((mutable_bitField0_ & 0x00008000) == 0x00008000)) {
           uninterpretedOption_ = java.util.Collections.unmodifiableList(uninterpretedOption_);
         }
         this.unknownFields = unknownFields.build();
@@ -18623,6 +18663,69 @@ public final class DescriptorProtos {
       }
     }
 
+    public static final int SWIFT_PREFIX_FIELD_NUMBER = 39;
+    private volatile java.lang.Object swiftPrefix_;
+    /**
+     * <pre>
+     * By default Swift generators will take the proto package and CamelCase it
+     * replacing '.' with underscore and use that to prefix the types/symbols
+     * defined. When this options is provided, they will use this value instead
+     * to prefix the types/symbols defined.
+     * </pre>
+     *
+     * <code>optional string swift_prefix = 39;</code>
+     */
+    public boolean hasSwiftPrefix() {
+      return ((bitField0_ & 0x00004000) == 0x00004000);
+    }
+    /**
+     * <pre>
+     * By default Swift generators will take the proto package and CamelCase it
+     * replacing '.' with underscore and use that to prefix the types/symbols
+     * defined. When this options is provided, they will use this value instead
+     * to prefix the types/symbols defined.
+     * </pre>
+     *
+     * <code>optional string swift_prefix = 39;</code>
+     */
+    public java.lang.String getSwiftPrefix() {
+      java.lang.Object ref = swiftPrefix_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          swiftPrefix_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <pre>
+     * By default Swift generators will take the proto package and CamelCase it
+     * replacing '.' with underscore and use that to prefix the types/symbols
+     * defined. When this options is provided, they will use this value instead
+     * to prefix the types/symbols defined.
+     * </pre>
+     *
+     * <code>optional string swift_prefix = 39;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getSwiftPrefixBytes() {
+      java.lang.Object ref = swiftPrefix_;
+      if (ref instanceof java.lang.String) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        swiftPrefix_ = b;
+        return b;
+      } else {
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
     public static final int UNINTERPRETED_OPTION_FIELD_NUMBER = 999;
     private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption> uninterpretedOption_;
     /**
@@ -18745,6 +18848,9 @@ public final class DescriptorProtos {
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 37, csharpNamespace_);
       }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 39, swiftPrefix_);
+      }
       for (int i = 0; i < uninterpretedOption_.size(); i++) {
         output.writeMessage(999, uninterpretedOption_.get(i));
       }
@@ -18808,6 +18914,9 @@ public final class DescriptorProtos {
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(37, csharpNamespace_);
       }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(39, swiftPrefix_);
+      }
       for (int i = 0; i < uninterpretedOption_.size(); i++) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(999, uninterpretedOption_.get(i));
@@ -18899,6 +19008,11 @@ public final class DescriptorProtos {
         result = result && getCsharpNamespace()
             .equals(other.getCsharpNamespace());
       }
+      result = result && (hasSwiftPrefix() == other.hasSwiftPrefix());
+      if (hasSwiftPrefix()) {
+        result = result && getSwiftPrefix()
+            .equals(other.getSwiftPrefix());
+      }
       result = result && getUninterpretedOptionList()
           .equals(other.getUninterpretedOptionList());
       result = result && unknownFields.equals(other.unknownFields);
@@ -18913,7 +19027,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasJavaPackage()) {
         hash = (37 * hash) + JAVA_PACKAGE_FIELD_NUMBER;
         hash = (53 * hash) + getJavaPackage().hashCode();
@@ -18978,6 +19092,10 @@ public final class DescriptorProtos {
         hash = (37 * hash) + CSHARP_NAMESPACE_FIELD_NUMBER;
         hash = (53 * hash) + getCsharpNamespace().hashCode();
       }
+      if (hasSwiftPrefix()) {
+        hash = (37 * hash) + SWIFT_PREFIX_FIELD_NUMBER;
+        hash = (53 * hash) + getSwiftPrefix().hashCode();
+      }
       if (getUninterpretedOptionCount() > 0) {
         hash = (37 * hash) + UNINTERPRETED_OPTION_FIELD_NUMBER;
         hash = (53 * hash) + getUninterpretedOptionList().hashCode();
@@ -19131,9 +19249,11 @@ public final class DescriptorProtos {
         bitField0_ = (bitField0_ & ~0x00001000);
         csharpNamespace_ = "";
         bitField0_ = (bitField0_ & ~0x00002000);
+        swiftPrefix_ = "";
+        bitField0_ = (bitField0_ & ~0x00004000);
         if (uninterpretedOptionBuilder_ == null) {
           uninterpretedOption_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00004000);
+          bitField0_ = (bitField0_ & ~0x00008000);
         } else {
           uninterpretedOptionBuilder_.clear();
         }
@@ -19217,10 +19337,14 @@ public final class DescriptorProtos {
           to_bitField0_ |= 0x00002000;
         }
         result.csharpNamespace_ = csharpNamespace_;
+        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+          to_bitField0_ |= 0x00004000;
+        }
+        result.swiftPrefix_ = swiftPrefix_;
         if (uninterpretedOptionBuilder_ == null) {
-          if (((bitField0_ & 0x00004000) == 0x00004000)) {
+          if (((bitField0_ & 0x00008000) == 0x00008000)) {
             uninterpretedOption_ = java.util.Collections.unmodifiableList(uninterpretedOption_);
-            bitField0_ = (bitField0_ & ~0x00004000);
+            bitField0_ = (bitField0_ & ~0x00008000);
           }
           result.uninterpretedOption_ = uninterpretedOption_;
         } else {
@@ -19343,11 +19467,16 @@ public final class DescriptorProtos {
           csharpNamespace_ = other.csharpNamespace_;
           onChanged();
         }
+        if (other.hasSwiftPrefix()) {
+          bitField0_ |= 0x00004000;
+          swiftPrefix_ = other.swiftPrefix_;
+          onChanged();
+        }
         if (uninterpretedOptionBuilder_ == null) {
           if (!other.uninterpretedOption_.isEmpty()) {
             if (uninterpretedOption_.isEmpty()) {
               uninterpretedOption_ = other.uninterpretedOption_;
-              bitField0_ = (bitField0_ & ~0x00004000);
+              bitField0_ = (bitField0_ & ~0x00008000);
             } else {
               ensureUninterpretedOptionIsMutable();
               uninterpretedOption_.addAll(other.uninterpretedOption_);
@@ -19360,7 +19489,7 @@ public final class DescriptorProtos {
               uninterpretedOptionBuilder_.dispose();
               uninterpretedOptionBuilder_ = null;
               uninterpretedOption_ = other.uninterpretedOption_;
-              bitField0_ = (bitField0_ & ~0x00004000);
+              bitField0_ = (bitField0_ & ~0x00008000);
               uninterpretedOptionBuilder_ = 
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getUninterpretedOptionFieldBuilder() : null;
@@ -20454,12 +20583,130 @@ public final class DescriptorProtos {
         return this;
       }
 
+      private java.lang.Object swiftPrefix_ = "";
+      /**
+       * <pre>
+       * By default Swift generators will take the proto package and CamelCase it
+       * replacing '.' with underscore and use that to prefix the types/symbols
+       * defined. When this options is provided, they will use this value instead
+       * to prefix the types/symbols defined.
+       * </pre>
+       *
+       * <code>optional string swift_prefix = 39;</code>
+       */
+      public boolean hasSwiftPrefix() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <pre>
+       * By default Swift generators will take the proto package and CamelCase it
+       * replacing '.' with underscore and use that to prefix the types/symbols
+       * defined. When this options is provided, they will use this value instead
+       * to prefix the types/symbols defined.
+       * </pre>
+       *
+       * <code>optional string swift_prefix = 39;</code>
+       */
+      public java.lang.String getSwiftPrefix() {
+        java.lang.Object ref = swiftPrefix_;
+        if (!(ref instanceof java.lang.String)) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            swiftPrefix_ = s;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <pre>
+       * By default Swift generators will take the proto package and CamelCase it
+       * replacing '.' with underscore and use that to prefix the types/symbols
+       * defined. When this options is provided, they will use this value instead
+       * to prefix the types/symbols defined.
+       * </pre>
+       *
+       * <code>optional string swift_prefix = 39;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getSwiftPrefixBytes() {
+        java.lang.Object ref = swiftPrefix_;
+        if (ref instanceof String) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          swiftPrefix_ = b;
+          return b;
+        } else {
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <pre>
+       * By default Swift generators will take the proto package and CamelCase it
+       * replacing '.' with underscore and use that to prefix the types/symbols
+       * defined. When this options is provided, they will use this value instead
+       * to prefix the types/symbols defined.
+       * </pre>
+       *
+       * <code>optional string swift_prefix = 39;</code>
+       */
+      public Builder setSwiftPrefix(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00004000;
+        swiftPrefix_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * By default Swift generators will take the proto package and CamelCase it
+       * replacing '.' with underscore and use that to prefix the types/symbols
+       * defined. When this options is provided, they will use this value instead
+       * to prefix the types/symbols defined.
+       * </pre>
+       *
+       * <code>optional string swift_prefix = 39;</code>
+       */
+      public Builder clearSwiftPrefix() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        swiftPrefix_ = getDefaultInstance().getSwiftPrefix();
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * By default Swift generators will take the proto package and CamelCase it
+       * replacing '.' with underscore and use that to prefix the types/symbols
+       * defined. When this options is provided, they will use this value instead
+       * to prefix the types/symbols defined.
+       * </pre>
+       *
+       * <code>optional string swift_prefix = 39;</code>
+       */
+      public Builder setSwiftPrefixBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00004000;
+        swiftPrefix_ = value;
+        onChanged();
+        return this;
+      }
+
       private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption> uninterpretedOption_ =
         java.util.Collections.emptyList();
       private void ensureUninterpretedOptionIsMutable() {
-        if (!((bitField0_ & 0x00004000) == 0x00004000)) {
+        if (!((bitField0_ & 0x00008000) == 0x00008000)) {
           uninterpretedOption_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption>(uninterpretedOption_);
-          bitField0_ |= 0x00004000;
+          bitField0_ |= 0x00008000;
          }
       }
 
@@ -20653,7 +20900,7 @@ public final class DescriptorProtos {
       public Builder clearUninterpretedOption() {
         if (uninterpretedOptionBuilder_ == null) {
           uninterpretedOption_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00004000);
+          bitField0_ = (bitField0_ & ~0x00008000);
           onChanged();
         } else {
           uninterpretedOptionBuilder_.clear();
@@ -20758,7 +21005,7 @@ public final class DescriptorProtos {
           uninterpretedOptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption, org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOptionOrBuilder>(
                   uninterpretedOption_,
-                  ((bitField0_ & 0x00004000) == 0x00004000),
+                  ((bitField0_ & 0x00008000) == 0x00008000),
                   getParentForChildren(),
                   isClean());
           uninterpretedOption_ = null;
@@ -21448,7 +21695,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasMessageSetWireFormat()) {
         hash = (37 * hash) + MESSAGE_SET_WIRE_FORMAT_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -23505,7 +23752,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCtype()) {
         hash = (37 * hash) + CTYPE_FIELD_NUMBER;
         hash = (53 * hash) + ctype_;
@@ -24957,7 +25204,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getUninterpretedOptionCount() > 0) {
         hash = (37 * hash) + UNINTERPRETED_OPTION_FIELD_NUMBER;
         hash = (53 * hash) + getUninterpretedOptionList().hashCode();
@@ -25998,7 +26245,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasAllowAlias()) {
         hash = (37 * hash) + ALLOW_ALIAS_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -27119,7 +27366,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDeprecated()) {
         hash = (37 * hash) + DEPRECATED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -28174,7 +28421,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDeprecated()) {
         hash = (37 * hash) + DEPRECATED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -28930,6 +29177,15 @@ public final class DescriptorProtos {
     boolean getDeprecated();
 
     /**
+     * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+     */
+    boolean hasIdempotencyLevel();
+    /**
+     * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel getIdempotencyLevel();
+
+    /**
      * <pre>
      * The parser stores options it doesn't recognize here. See above.
      * </pre>
@@ -28987,6 +29243,7 @@ public final class DescriptorProtos {
     }
     private MethodOptions() {
       deprecated_ = false;
+      idempotencyLevel_ = 0;
       uninterpretedOption_ = java.util.Collections.emptyList();
     }
 
@@ -29023,10 +29280,21 @@ public final class DescriptorProtos {
               deprecated_ = input.readBool();
               break;
             }
+            case 272: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel value = org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(34, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                idempotencyLevel_ = rawValue;
+              }
+              break;
+            }
             case 7994: {
-              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
                 uninterpretedOption_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption>();
-                mutable_bitField0_ |= 0x00000002;
+                mutable_bitField0_ |= 0x00000004;
               }
               uninterpretedOption_.add(
                   input.readMessage(org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption.PARSER, extensionRegistry));
@@ -29040,7 +29308,7 @@ public final class DescriptorProtos {
         throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
           uninterpretedOption_ = java.util.Collections.unmodifiableList(uninterpretedOption_);
         }
         this.unknownFields = unknownFields.build();
@@ -29059,6 +29327,127 @@ public final class DescriptorProtos {
               org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.class, org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.Builder.class);
     }
 
+    /**
+     * <pre>
+     * Is this method side-effect-free (or safe in HTTP parlance), or idempotent,
+     * or neither? HTTP based RPC implementation may choose GET verb for safe
+     * methods, and PUT verb for idempotent methods instead of the default POST.
+     * </pre>
+     *
+     * Protobuf enum {@code google.protobuf.MethodOptions.IdempotencyLevel}
+     */
+    public enum IdempotencyLevel
+        implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+      /**
+       * <code>IDEMPOTENCY_UNKNOWN = 0;</code>
+       */
+      IDEMPOTENCY_UNKNOWN(0),
+      /**
+       * <pre>
+       * implies idempotent
+       * </pre>
+       *
+       * <code>NO_SIDE_EFFECTS = 1;</code>
+       */
+      NO_SIDE_EFFECTS(1),
+      /**
+       * <pre>
+       * idempotent, but may have side effects
+       * </pre>
+       *
+       * <code>IDEMPOTENT = 2;</code>
+       */
+      IDEMPOTENT(2),
+      ;
+
+      /**
+       * <code>IDEMPOTENCY_UNKNOWN = 0;</code>
+       */
+      public static final int IDEMPOTENCY_UNKNOWN_VALUE = 0;
+      /**
+       * <pre>
+       * implies idempotent
+       * </pre>
+       *
+       * <code>NO_SIDE_EFFECTS = 1;</code>
+       */
+      public static final int NO_SIDE_EFFECTS_VALUE = 1;
+      /**
+       * <pre>
+       * idempotent, but may have side effects
+       * </pre>
+       *
+       * <code>IDEMPOTENT = 2;</code>
+       */
+      public static final int IDEMPOTENT_VALUE = 2;
+
+
+      public final int getNumber() {
+        return value;
+      }
+
+      /**
+       * @deprecated Use {@link #forNumber(int)} instead.
+       */
+      @java.lang.Deprecated
+      public static IdempotencyLevel valueOf(int value) {
+        return forNumber(value);
+      }
+
+      public static IdempotencyLevel forNumber(int value) {
+        switch (value) {
+          case 0: return IDEMPOTENCY_UNKNOWN;
+          case 1: return NO_SIDE_EFFECTS;
+          case 2: return IDEMPOTENT;
+          default: return null;
+        }
+      }
+
+      public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<IdempotencyLevel>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+          IdempotencyLevel> internalValueMap =
+            new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<IdempotencyLevel>() {
+              public IdempotencyLevel findValueByNumber(int number) {
+                return IdempotencyLevel.forNumber(number);
+              }
+            };
+
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(ordinal());
+      }
+      public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.getDescriptor().getEnumTypes().get(0);
+      }
+
+      private static final IdempotencyLevel[] VALUES = values();
+
+      public static IdempotencyLevel valueOf(
+          org.apache.hadoop.hbase.shaded.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 value;
+
+      private IdempotencyLevel(int value) {
+        this.value = value;
+      }
+
+      // @@protoc_insertion_point(enum_scope:google.protobuf.MethodOptions.IdempotencyLevel)
+    }
+
     private int bitField0_;
     public static final int DEPRECATED_FIELD_NUMBER = 33;
     private boolean deprecated_;
@@ -29089,6 +29478,22 @@ public final class DescriptorProtos {
       return deprecated_;
     }
 
+    public static final int IDEMPOTENCY_LEVEL_FIELD_NUMBER = 34;
+    private int idempotencyLevel_;
+    /**
+     * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+     */
+    public boolean hasIdempotencyLevel() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel getIdempotencyLevel() {
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel result = org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel.valueOf(idempotencyLevel_);
+      return result == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel.IDEMPOTENCY_UNKNOWN : result;
+    }
+
     public static final int UNINTERPRETED_OPTION_FIELD_NUMBER = 999;
     private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption> uninterpretedOption_;
     /**
@@ -29172,6 +29577,9 @@ public final class DescriptorProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(33, deprecated_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeEnum(34, idempotencyLevel_);
+      }
       for (int i = 0; i < uninterpretedOption_.size(); i++) {
         output.writeMessage(999, uninterpretedOption_.get(i));
       }
@@ -29188,6 +29596,10 @@ public final class DescriptorProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(33, deprecated_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeEnumSize(34, idempotencyLevel_);
+      }
       for (int i = 0; i < uninterpretedOption_.size(); i++) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(999, uninterpretedOption_.get(i));
@@ -29215,6 +29627,10 @@ public final class DescriptorProtos {
         result = result && (getDeprecated()
             == other.getDeprecated());
       }
+      result = result && (hasIdempotencyLevel() == other.hasIdempotencyLevel());
+      if (hasIdempotencyLevel()) {
+        result = result && idempotencyLevel_ == other.idempotencyLevel_;
+      }
       result = result && getUninterpretedOptionList()
           .equals(other.getUninterpretedOptionList());
       result = result && unknownFields.equals(other.unknownFields);
@@ -29229,12 +29645,16 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDeprecated()) {
         hash = (37 * hash) + DEPRECATED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getDeprecated());
       }
+      if (hasIdempotencyLevel()) {
+        hash = (37 * hash) + IDEMPOTENCY_LEVEL_FIELD_NUMBER;
+        hash = (53 * hash) + idempotencyLevel_;
+      }
       if (getUninterpretedOptionCount() > 0) {
         hash = (37 * hash) + UNINTERPRETED_OPTION_FIELD_NUMBER;
         hash = (53 * hash) + getUninterpretedOptionList().hashCode();
@@ -29362,9 +29782,11 @@ public final class DescriptorProtos {
         super.clear();
         deprecated_ = false;
         bitField0_ = (bitField0_ & ~0x00000001);
+        idempotencyLevel_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
         if (uninterpretedOptionBuilder_ == null) {
           uninterpretedOption_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
+          bitField0_ = (bitField0_ & ~0x00000004);
         } else {
           uninterpretedOptionBuilder_.clear();
         }
@@ -29396,10 +29818,14 @@ public final class DescriptorProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.deprecated_ = deprecated_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.idempotencyLevel_ = idempotencyLevel_;
         if (uninterpretedOptionBuilder_ == null) {
-          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
             uninterpretedOption_ = java.util.Collections.unmodifiableList(uninterpretedOption_);
-            bitField0_ = (bitField0_ & ~0x00000002);
+            bitField0_ = (bitField0_ & ~0x00000004);
           }
           result.uninterpretedOption_ = uninterpretedOption_;
         } else {
@@ -29473,11 +29899,14 @@ public final class DescriptorProtos {
         if (other.hasDeprecated()) {
           setDeprecated(other.getDeprecated());
         }
+        if (other.hasIdempotencyLevel()) {
+          setIdempotencyLevel(other.getIdempotencyLevel());
+        }
         if (uninterpretedOptionBuilder_ == null) {
           if (!other.uninterpretedOption_.isEmpty()) {
             if (uninterpretedOption_.isEmpty()) {
               uninterpretedOption_ = other.uninterpretedOption_;
-              bitField0_ = (bitField0_ & ~0x00000002);
+              bitField0_ = (bitField0_ & ~0x00000004);
             } else {
               ensureUninterpretedOptionIsMutable();
               uninterpretedOption_.addAll(other.uninterpretedOption_);
@@ -29490,7 +29919,7 @@ public final class DescriptorProtos {
               uninterpretedOptionBuilder_.dispose();
               uninterpretedOptionBuilder_ = null;
               uninterpretedOption_ = other.uninterpretedOption_;
-              bitField0_ = (bitField0_ & ~0x00000002);
+              bitField0_ = (bitField0_ & ~0x00000004);
               uninterpretedOptionBuilder_ = 
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getUninterpretedOptionFieldBuilder() : null;
@@ -29596,12 +30025,48 @@ public final class DescriptorProtos {
         return this;
       }
 
+      private int idempotencyLevel_ = 0;
+      /**
+       * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+       */
+      public boolean hasIdempotencyLevel() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel getIdempotencyLevel() {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel result = org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel.valueOf(idempotencyLevel_);
+        return result == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel.IDEMPOTENCY_UNKNOWN : result;
+      }
+      /**
+       * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+       */
+      public Builder setIdempotencyLevel(org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.MethodOptions.IdempotencyLevel value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000002;
+        idempotencyLevel_ = value.getNumber();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .google.protobuf.MethodOptions.IdempotencyLevel idempotency_level = 34 [default = IDEMPOTENCY_UNKNOWN];</code>
+       */
+      public Builder clearIdempotencyLevel() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        idempotencyLevel_ = 0;
+        onChanged();
+        return this;
+      }
+
       private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption> uninterpretedOption_ =
         java.util.Collections.emptyList();
       private void ensureUninterpretedOptionIsMutable() {
-        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
           uninterpretedOption_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption>(uninterpretedOption_);
-          bitField0_ |= 0x00000002;
+          bitField0_ |= 0x00000004;
          }
       }
 
@@ -29795,7 +30260,7 @@ public final class DescriptorProtos {
       public Builder clearUninterpretedOption() {
         if (uninterpretedOptionBuilder_ == null) {
           uninterpretedOption_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
+          bitField0_ = (bitField0_ & ~0x00000004);
           onChanged();
         } else {
           uninterpretedOptionBuilder_.clear();
@@ -29900,7 +30365,7 @@ public final class DescriptorProtos {
           uninterpretedOptionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption, org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOption.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.DescriptorProtos.UninterpretedOptionOrBuilder>(
                   uninterpretedOption_,
-                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  ((bitField0_ & 0x00000004) == 0x00000004),
                   getParentForChildren(),
                   isClean());
           uninterpretedOption_ = null;
@@ -30439,7 +30904,7 @@ public final class DescriptorProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasNamePart()) {
           hash = (37 * hash) + NAME_PART_FIELD_NUMBER;
           hash = (53 * hash) + getNamePart().hashCode();
@@ -31180,7 +31645,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getNameCount() > 0) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getNameList().hashCode();
@@ -33406,7 +33871,7 @@ public final class DescriptorProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (getPathCount() > 0) {
           hash = (37 * hash) + PATH_FIELD_NUMBER;
           hash = (53 * hash) + getPathList().hashCode();
@@ -34906,7 +35371,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getLocationCount() > 0) {
         hash = (37 * hash) + LOCATION_FIELD_NUMBER;
         hash = (53 * hash) + getLocationList().hashCode();
@@ -36845,7 +37310,7 @@ public final class DescriptorProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (getPathCount() > 0) {
           hash = (37 * hash) + PATH_FIELD_NUMBER;
           hash = (53 * hash) + getPathList().hashCode();
@@ -37593,7 +38058,7 @@ public final class DescriptorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getAnnotationCount() > 0) {
         hash = (37 * hash) + ANNOTATION_FIELD_NUMBER;
         hash = (53 * hash) + getAnnotationList().hashCode();
@@ -38433,7 +38898,7 @@ public final class DescriptorProtos {
       "\n\013output_type\030\003 \001(\t\022/\n\007options\030\004 \001(\0132\036.g" +
       "oogle.protobuf.MethodOptions\022\037\n\020client_s" +
       "treaming\030\005 \001(\010:\005false\022\037\n\020server_streamin" +
-      "g\030\006 \001(\010:\005false\"\204\005\n\013FileOptions\022\024\n\014java_p" +
+      "g\030\006 \001(\010:\005false\"\232\005\n\013FileOptions\022\024\n\014java_p" +
       "ackage\030\001 \001(\t\022\034\n\024java_outer_classname\030\010 \001" +
       "(\t\022\"\n\023java_multiple_files\030\n \001(\010:\005false\022)" +
       "\n\035java_generate_equals_and_hash\030\024 \001(\010B\002\030" +
@@ -38445,62 +38910,68 @@ public final class DescriptorProtos {
       "\005false\022\"\n\023py_generic_services\030\022 \001(\010:\005fal" +
       "se\022\031\n\ndeprecated\030\027 \001(\010:\005false\022\037\n\020cc_enab" +
       "le_arenas\030\037 \001(\010:\005false\022\031\n\021objc_class_pre" +
-      "fix\030$ \001(\t\022\030\n\020csharp_namespace\030% \001(\t\022C\n\024u" +
-      "ninterpreted_option\030\347\007 \003(\0132$.google.prot" +
-      "obuf.UninterpretedOption\":\n\014OptimizeMode",
-      "\022\t\n\005SPEED\020\001\022\r\n\tCODE_SIZE\020\002\022\020\n\014LITE_RUNTI" +
-      "ME\020\003*\t\010\350\007\020\200\200\200\200\002J\004\010&\020\'\"\354\001\n\016MessageOptions" +
-      "\022&\n\027message_set_wire_format\030\001 \001(\010:\005false" +
-      "\022.\n\037no_standard_descriptor_accessor\030\002 \001(" +
-      "\010:\005false\022\031\n\ndeprecated\030\003 \001(\010:\005false\022\021\n\tm" +
-      "ap_entry\030\007 \001(\010\022C\n\024uninterpreted_option\030\347" +
-      "\007 \003(\0132$.google.protobuf.UninterpretedOpt" +
-      "ion*\t\010\350\007\020\200\200\200\200\002J\004\010\010\020\t\"\236\003\n\014FieldOptions\022:\n" +
-      "\005ctype\030\001 \001(\0162#.google.protobuf.FieldOpti" +
-      "ons.CType:\006STRING\022\016\n\006packed\030\002 \001(\010\022?\n\006jst",
-      "ype\030\006 \001(\0162$.google.protobuf.FieldOptions" +
-      ".JSType:\tJS_NORMAL\022\023\n\004lazy\030\005 \001(\010:\005false\022" +
-      "\031\n\ndeprecated\030\003 \001(\010:\005false\022\023\n\004weak\030\n \001(\010" +
-      ":\005false\022C\n\024uninterpreted_option\030\347\007 \003(\0132$" +
-      ".google.protobuf.UninterpretedOption\"/\n\005" +
-      "CType\022\n\n\006STRING\020\000\022\010\n\004CORD\020\001\022\020\n\014STRING_PI" +
-      "ECE\020\002\"5\n\006JSType\022\r\n\tJS_NORMAL\020\000\022\r\n\tJS_STR" +
-      "ING\020\001\022\r\n\tJS_NUMBER\020\002*\t\010\350\007\020\200\200\200\200\002J\004\010\004\020\005\"^\n" +
-      "\014OneofOptions\022C\n\024uninterpreted_option\030\347\007" +
-      " \003(\0132$.google.protobuf.UninterpretedOpti",
-      "on*\t\010\350\007\020\200\200\200\200\002\"\215\001\n\013EnumOptions\022\023\n\013allow_a" +
-      "lias\030\002 \001(\010\022\031\n\ndeprecated\030\003 \001(\010:\005false\022C\n" +
-      "\024uninterpreted_option\030\347\007 \003(\0132$.google.pr" +
-      "otobuf.UninterpretedOption*\t\010\350\007\020\200\200\200\200\002\"}\n" +
-      "\020EnumValueOptions\022\031\n\ndeprecated\030\001 \001(\010:\005f" +
-      "alse\022C\n\024uninterpreted_option\030\347\007 \003(\0132$.go" +
-      "ogle.protobuf.UninterpretedOption*\t\010\350\007\020\200" +
-      "\200\200\200\002\"{\n\016ServiceOptions\022\031\n\ndeprecated\030! \001" +
-      "(\010:\005false\022C\n\024uninterpreted_option\030\347\007 \003(\013" +
-      "2$.google.protobuf.UninterpretedOption*\t",
-      "\010\350\007\020\200\200\200\200\002\"z\n\rMethodOptions\022\031\n\ndeprecated" +
-      "\030! \001(\010:\005false\022C\n\024uninterpreted_option\030\347\007" +
+      "fix\030$ \001(\t\022\030\n\020csharp_namespace\030% \001(\t\022\024\n\014s" +
+      "wift_prefix\030\' \001(\t\022C\n\024uninterpreted_optio" +
+      "n\030\347\007 \003(\0132$.google.protobuf.Uninterpreted",
+      "Option\":\n\014OptimizeMode\022\t\n\005SPEED\020\001\022\r\n\tCOD" +
+      "E_SIZE\020\002\022\020\n\014LITE_RUNTIME\020\003*\t\010\350\007\020\200\200\200\200\002J\004\010" +
+      "&\020\'\"\354\001\n\016MessageOptions\022&\n\027message_set_wi" +
+      "re_format\030\001 \001(\010:\005false\022.\n\037no_standard_de" +
+      "scriptor_accessor\030\002 \001(\010:\005false\022\031\n\ndeprec" +
+      "ated\030\003 \001(\010:\005false\022\021\n\tmap_entry\030\007 \001(\010\022C\n\024" +
+      "uninterpreted_option\030\347\007 \003(\0132$.google.pro" +
+      "tobuf.UninterpretedOption*\t\010\350\007\020\200\200\200\200\002J\004\010\010" +
+      "\020\t\"\236\003\n\014FieldOptions\022:\n\005ctype\030\001 \001(\0162#.goo" +
+      "gle.protobuf.FieldOptions.CType:\006STRING\022",
+      "\016\n\006packed\030\002 \001(\010\022?\n\006jstype\030\006 \001(\0162$.google" +
+      ".protobuf.FieldOptions.JSType:\tJS_NORMAL" +
+      "\022\023\n\004lazy\030\005 \001(\010:\005false\022\031\n\ndeprecated\030\003 \001(" +
+      "\010:\005false\022\023\n\004weak\030\n \001(\010:\005false\022C\n\024uninter" +
+      "preted_option\030\347\007 \003(\0132$.google.protobuf.U" +
+      "ninterpretedOption\"/\n\005CType\022\n\n\006STRING\020\000\022" +
+      "\010\n\004CORD\020\001\022\020\n\014STRING_PIECE\020\002\"5\n\006JSType\022\r\n" +
+      "\tJS_NORMAL\020\000\022\r\n\tJS_STRING\020\001\022\r\n\tJS_NUMBER" +
+      "\020\002*\t\010\350\007\020\200\200\200\200\002J\004\010\004\020\005\"^\n\014OneofOptions\022C\n\024u" +
+      "ninterpreted_option\030\347\007 \003(\0132$.google.prot",
+      "obuf.UninterpretedOption*\t\010\350\007\020\200\200\200\200\002\"\215\001\n\013" +
+      "EnumOptions\022\023\n\013allow_alias\030\002 \001(\010\022\031\n\ndepr" +
+      "ecated\030\003 \001(\010:\005false\022C\n\024uninterpreted_opt" +
+      "ion\030\347\007 \003(\0132$.google.protobuf.Uninterpret" +
+      "edOption*\t\010\350\007\020\200\200\200\200\002\"}\n\020EnumValueOptions\022" +
+      "\031\n\ndeprecated\030\001 \001(\010:\005false\022C\n\024uninterpre" +
+      "ted_option\030\347\007 \003(\0132$.google.protobuf.Unin" +
+      "terpretedOption*\t\010\350\007\020\200\200\200\200\002\"{\n\016ServiceOpt" +
+      "ions\022\031\n\ndeprecated\030! \001(\010:\005false\022C\n\024unint" +
+      "erpreted_option\030\347\007 \003(\0132$.google.protobuf",
+      ".UninterpretedOption*\t\010\350\007\020\200\200\200\200\002\"\255\002\n\rMeth" +
+      "odOptions\022\031\n\ndeprecated\030! \001(\010:\005false\022_\n\021" +
+      "idempotency_level\030\" \001(\0162/.google.protobu" +
+      "f.MethodOptions.IdempotencyLevel:\023IDEMPO" +
+      "TENCY_UNKNOWN\022C\n\024uninterpreted_option\030\347\007" +
       " \003(\0132$.google.protobuf.UninterpretedOpti" +
-      "on*\t\010\350\007\020\200\200\200\200\002\"\236\002\n\023UninterpretedOption\022;\n" +
-      "\004name\030\002 \003(\0132-.google.protobuf.Uninterpre" +
-      "tedOption.NamePart\022\030\n\020identifier_value\030\003" +
-      " \001(\t\022\032\n\022positive_int_value\030\004 \001(\004\022\032\n\022nega" +
-      "tive_int_value\030\005 \001(\003\022\024\n\014double_value\030\006 \001" +
-      "(\001\022\024\n\014string_value\030\007 \001(\014\022\027\n\017aggregate_va" +
-      "lue\030\010 \001(\t\0323\n\010NamePart\022\021\n\tname_part\030\001 \002(\t",
-      "\022\024\n\014is_extension\030\002 \002(\010\"\325\001\n\016SourceCodeInf" +
-      "o\022:\n\010location\030\001 \003(\0132(.google.protobuf.So" +
-      "urceCodeInfo.Location\032\206\001\n\010Location\022\020\n\004pa" +
-      "th\030\001 \003(\005B\002\020\001\022\020\n\004span\030\002 \003(\005B\002\020\001\022\030\n\020leadin" +
-      "g_comments\030\003 \001(\t\022\031\n\021trailing_comments\030\004 " +
-      "\001(\t\022!\n\031leading_detached_comments\030\006 \003(\t\"\247" +
-      "\001\n\021GeneratedCodeInfo\022A\n\nannotation\030\001 \003(\013" +
-      "2-.google.protobuf.GeneratedCodeInfo.Ann" +
-      "otation\032O\n\nAnnotation\022\020\n\004path\030\001 \003(\005B\002\020\001\022" +
-      "\023\n\013source_file\030\002 \001(\t\022\r\n\005begin\030\003 \001(\005\022\013\n\003e",
-      "nd\030\004 \001(\005BX\n\023com.google.protobufB\020Descrip" +
-      "torProtosH\001Z\ndescriptor\242\002\003GPB\252\002\032Google.P" +
-      "rotobuf.Reflection"
+      "on\"P\n\020IdempotencyLevel\022\027\n\023IDEMPOTENCY_UN" +
+      "KNOWN\020\000\022\023\n\017NO_SIDE_EFFECTS\020\001\022\016\n\nIDEMPOTE" +
+      "NT\020\002*\t\010\350\007\020\200\200\200\200\002\"\236\002\n\023UninterpretedOption\022" +
+      ";\n\004name\030\002 \003(\0132-.google.protobuf.Uninterp",
+      "retedOption.NamePart\022\030\n\020identifier_value" +
+      "\030\003 \001(\t\022\032\n\022positive_int_value\030\004 \001(\004\022\032\n\022ne" +
+      "gative_int_value\030\005 \001(\003\022\024\n\014double_value\030\006" +
+      " \001(\001\022\024\n\014string_value\030\007 \001(\014\022\027\n\017aggregate_" +
+      "value\030\010 \001(\t\0323\n\010NamePart\022\021\n\tname_part\030\001 \002" +
+      "(\t\022\024\n\014is_extension\030\002 \002(\010\"\325\001\n\016SourceCodeI" +
+      "nfo\022:\n\010location\030\001 \003(\0132(.google.protobuf." +
+      "SourceCodeInfo.Location\032\206\001\n\010Location\022\020\n\004" +
+      "path\030\001 \003(\005B\002\020\001\022\020\n\004span\030\002 \003(\005B\002\020\001\022\030\n\020lead" +
+      "ing_comments\030\003 \001(\t\022\031\n\021trailing_comments\030",
+      "\004 \001(\t\022!\n\031leading_detached_comments\030\006 \003(\t" +
+      "\"\247\001\n\021GeneratedCodeInfo\022A\n\nannotation\030\001 \003" +
+      "(\0132-.google.protobuf.GeneratedCodeInfo.A" +
+      "nnotation\032O\n\nAnnotation\022\020\n\004path\030\001 \003(\005B\002\020" +
+      "\001\022\023\n\013source_file\030\002 \001(\t\022\r\n\005begin\030\003 \001(\005\022\013\n" +
+      "\003end\030\004 \001(\005B\214\001\n\023com.google.protobufB\020Desc" +
+      "riptorProtosH\001Z>github.com/golang/protob" +
+      "uf/protoc-gen-go/descriptor;descriptor\242\002" +
+      "\003GPB\252\002\032Google.Protobuf.Reflection"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -38585,7 +39056,7 @@ public final class DescriptorProtos {
     internal_static_google_protobuf_FileOptions_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_google_protobuf_FileOptions_descriptor,
-        new java.lang.String[] { "JavaPackage", "JavaOuterClassname", "JavaMultipleFiles", "JavaGenerateEqualsAndHash", "JavaStringCheckUtf8", "OptimizeFor", "GoPackage", "CcGenericServices", "JavaGenericServices", "PyGenericServices", "Deprecated", "CcEnableArenas", "ObjcClassPrefix", "CsharpNamespace", "UninterpretedOption", });
+        new java.lang.String[] { "JavaPackage", "JavaOuterClassname", "JavaMultipleFiles", "JavaGenerateEqualsAndHash", "JavaStringCheckUtf8", "OptimizeFor", "GoPackage", "CcGenericServices", "JavaGenericServices", "PyGenericServices", "Deprecated", "CcEnableArenas", "ObjcClassPrefix", "CsharpNamespace", "SwiftPrefix", "UninterpretedOption", });
     internal_static_google_protobuf_MessageOptions_descriptor =
       getDescriptor().getMessageTypes().get(10);
     internal_static_google_protobuf_MessageOptions_fieldAccessorTable = new
@@ -38627,7 +39098,7 @@ public final class DescriptorProtos {
     internal_static_google_protobuf_MethodOptions_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_google_protobuf_MethodOptions_descriptor,
-        new java.lang.String[] { "Deprecated", "UninterpretedOption", });
+        new java.lang.String[] { "Deprecated", "IdempotencyLevel", "UninterpretedOption", });
     internal_static_google_protobuf_UninterpretedOption_descriptor =
       getDescriptor().getMessageTypes().get(17);
     internal_static_google_protobuf_UninterpretedOption_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Descriptors.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Descriptors.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Descriptors.java
index abc46b1..62ccd19 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Descriptors.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Descriptors.java
@@ -2123,7 +2123,7 @@ public final class Descriptors {
           // Can't happen, because addPackage() only fails when the name
           // conflicts with a non-package, but we have not yet added any
           // non-packages at this point.
-          assert false;
+          throw new AssertionError(e);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValue.java
index 2450436..98a0ecc 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValue.java
@@ -83,7 +83,7 @@ public  final class DoubleValue extends
    * The double value.
    * </pre>
    *
-   * <code>optional double value = 1;</code>
+   * <code>double value = 1;</code>
    */
   public double getValue() {
     return value_;
@@ -144,7 +144,7 @@ public  final class DoubleValue extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
         java.lang.Double.doubleToLongBits(getValue()));
@@ -372,7 +372,7 @@ public  final class DoubleValue extends
      * The double value.
      * </pre>
      *
-     * <code>optional double value = 1;</code>
+     * <code>double value = 1;</code>
      */
     public double getValue() {
       return value_;
@@ -382,7 +382,7 @@ public  final class DoubleValue extends
      * The double value.
      * </pre>
      *
-     * <code>optional double value = 1;</code>
+     * <code>double value = 1;</code>
      */
     public Builder setValue(double value) {
       
@@ -395,7 +395,7 @@ public  final class DoubleValue extends
      * The double value.
      * </pre>
      *
-     * <code>optional double value = 1;</code>
+     * <code>double value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValueOrBuilder.java
index 5bd1a3f..180fa50 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DoubleValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface DoubleValueOrBuilder extends
    * The double value.
    * </pre>
    *
-   * <code>optional double value = 1;</code>
+   * <code>double value = 1;</code>
    */
   double getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Duration.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Duration.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Duration.java
index a675f7b..c6736ef 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Duration.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Duration.java
@@ -124,7 +124,7 @@ public  final class Duration extends
    * to +315,576,000,000 inclusive.
    * </pre>
    *
-   * <code>optional int64 seconds = 1;</code>
+   * <code>int64 seconds = 1;</code>
    */
   public long getSeconds() {
     return seconds_;
@@ -142,7 +142,7 @@ public  final class Duration extends
    * to +999,999,999 inclusive.
    * </pre>
    *
-   * <code>optional int32 nanos = 2;</code>
+   * <code>int32 nanos = 2;</code>
    */
   public int getNanos() {
     return nanos_;
@@ -210,7 +210,7 @@ public  final class Duration extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + SECONDS_FIELD_NUMBER;
     hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
         getSeconds());
@@ -481,7 +481,7 @@ public  final class Duration extends
      * to +315,576,000,000 inclusive.
      * </pre>
      *
-     * <code>optional int64 seconds = 1;</code>
+     * <code>int64 seconds = 1;</code>
      */
     public long getSeconds() {
       return seconds_;
@@ -492,7 +492,7 @@ public  final class Duration extends
      * to +315,576,000,000 inclusive.
      * </pre>
      *
-     * <code>optional int64 seconds = 1;</code>
+     * <code>int64 seconds = 1;</code>
      */
     public Builder setSeconds(long value) {
       
@@ -506,7 +506,7 @@ public  final class Duration extends
      * to +315,576,000,000 inclusive.
      * </pre>
      *
-     * <code>optional int64 seconds = 1;</code>
+     * <code>int64 seconds = 1;</code>
      */
     public Builder clearSeconds() {
       
@@ -526,7 +526,7 @@ public  final class Duration extends
      * to +999,999,999 inclusive.
      * </pre>
      *
-     * <code>optional int32 nanos = 2;</code>
+     * <code>int32 nanos = 2;</code>
      */
     public int getNanos() {
       return nanos_;
@@ -541,7 +541,7 @@ public  final class Duration extends
      * to +999,999,999 inclusive.
      * </pre>
      *
-     * <code>optional int32 nanos = 2;</code>
+     * <code>int32 nanos = 2;</code>
      */
     public Builder setNanos(int value) {
       
@@ -559,7 +559,7 @@ public  final class Duration extends
      * to +999,999,999 inclusive.
      * </pre>
      *
-     * <code>optional int32 nanos = 2;</code>
+     * <code>int32 nanos = 2;</code>
      */
     public Builder clearNanos() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DurationOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DurationOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DurationOrBuilder.java
index 245f6b9..9c19aae 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DurationOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DurationOrBuilder.java
@@ -13,7 +13,7 @@ public interface DurationOrBuilder extends
    * to +315,576,000,000 inclusive.
    * </pre>
    *
-   * <code>optional int64 seconds = 1;</code>
+   * <code>int64 seconds = 1;</code>
    */
   long getSeconds();
 
@@ -27,7 +27,7 @@ public interface DurationOrBuilder extends
    * to +999,999,999 inclusive.
    * </pre>
    *
-   * <code>optional int32 nanos = 2;</code>
+   * <code>int32 nanos = 2;</code>
    */
   int getNanos();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DynamicMessage.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DynamicMessage.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DynamicMessage.java
index 110f2a9..13bccd6 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DynamicMessage.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/DynamicMessage.java
@@ -297,7 +297,7 @@ public final class DynamicMessage extends AbstractMessage {
         } catch (InvalidProtocolBufferException e) {
           throw e.setUnfinishedMessage(builder.buildPartial());
         } catch (IOException e) {
-          throw new InvalidProtocolBufferException(e.getMessage())
+          throw new InvalidProtocolBufferException(e)
               .setUnfinishedMessage(builder.buildPartial());
         }
         return builder.buildPartial();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Empty.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Empty.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Empty.java
index de6d523..61f1574 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Empty.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Empty.java
@@ -118,7 +118,7 @@ public  final class Empty extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (29 * hash) + unknownFields.hashCode();
     memoizedHashCode = hash;
     return hash;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Enum.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Enum.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Enum.java
index 6355bb3..3d2a206 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Enum.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Enum.java
@@ -130,7 +130,7 @@ public  final class Enum extends
    * Enum type name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -149,7 +149,7 @@ public  final class Enum extends
    * Enum type name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -282,7 +282,7 @@ public  final class Enum extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+   * <code>.google.protobuf.SourceContext source_context = 4;</code>
    */
   public boolean hasSourceContext() {
     return sourceContext_ != null;
@@ -292,7 +292,7 @@ public  final class Enum extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+   * <code>.google.protobuf.SourceContext source_context = 4;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext() {
     return sourceContext_ == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.getDefaultInstance() : sourceContext_;
@@ -302,7 +302,7 @@ public  final class Enum extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+   * <code>.google.protobuf.SourceContext source_context = 4;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder() {
     return getSourceContext();
@@ -315,7 +315,7 @@ public  final class Enum extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+   * <code>.google.protobuf.Syntax syntax = 5;</code>
    */
   public int getSyntaxValue() {
     return syntax_;
@@ -325,7 +325,7 @@ public  final class Enum extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+   * <code>.google.protobuf.Syntax syntax = 5;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -422,7 +422,7 @@ public  final class Enum extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + NAME_FIELD_NUMBER;
     hash = (53 * hash) + getName().hashCode();
     if (getEnumvalueCount() > 0) {
@@ -771,7 +771,7 @@ public  final class Enum extends
      * Enum type name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -790,7 +790,7 @@ public  final class Enum extends
      * Enum type name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -810,7 +810,7 @@ public  final class Enum extends
      * Enum type name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -827,7 +827,7 @@ public  final class Enum extends
      * Enum type name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder clearName() {
       
@@ -840,7 +840,7 @@ public  final class Enum extends
      * Enum type name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -1486,7 +1486,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public boolean hasSourceContext() {
       return sourceContextBuilder_ != null || sourceContext_ != null;
@@ -1496,7 +1496,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext() {
       if (sourceContextBuilder_ == null) {
@@ -1510,7 +1510,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public Builder setSourceContext(org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext value) {
       if (sourceContextBuilder_ == null) {
@@ -1530,7 +1530,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public Builder setSourceContext(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder builderForValue) {
@@ -1548,7 +1548,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public Builder mergeSourceContext(org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext value) {
       if (sourceContextBuilder_ == null) {
@@ -1570,7 +1570,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public Builder clearSourceContext() {
       if (sourceContextBuilder_ == null) {
@@ -1588,7 +1588,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder getSourceContextBuilder() {
       
@@ -1600,7 +1600,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder() {
       if (sourceContextBuilder_ != null) {
@@ -1615,7 +1615,7 @@ public  final class Enum extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+     * <code>.google.protobuf.SourceContext source_context = 4;</code>
      */
     private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
         org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext, org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder> 
@@ -1637,7 +1637,7 @@ public  final class Enum extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+     * <code>.google.protobuf.Syntax syntax = 5;</code>
      */
     public int getSyntaxValue() {
       return syntax_;
@@ -1647,7 +1647,7 @@ public  final class Enum extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+     * <code>.google.protobuf.Syntax syntax = 5;</code>
      */
     public Builder setSyntaxValue(int value) {
       syntax_ = value;
@@ -1659,7 +1659,7 @@ public  final class Enum extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+     * <code>.google.protobuf.Syntax syntax = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -1670,7 +1670,7 @@ public  final class Enum extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+     * <code>.google.protobuf.Syntax syntax = 5;</code>
      */
     public Builder setSyntax(org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax value) {
       if (value == null) {
@@ -1686,7 +1686,7 @@ public  final class Enum extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+     * <code>.google.protobuf.Syntax syntax = 5;</code>
      */
     public Builder clearSyntax() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumOrBuilder.java
index af5e01e..6e2d7d4 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumOrBuilder.java
@@ -12,7 +12,7 @@ public interface EnumOrBuilder extends
    * Enum type name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   java.lang.String getName();
   /**
@@ -20,7 +20,7 @@ public interface EnumOrBuilder extends
    * Enum type name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
@@ -118,7 +118,7 @@ public interface EnumOrBuilder extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+   * <code>.google.protobuf.SourceContext source_context = 4;</code>
    */
   boolean hasSourceContext();
   /**
@@ -126,7 +126,7 @@ public interface EnumOrBuilder extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+   * <code>.google.protobuf.SourceContext source_context = 4;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext();
   /**
@@ -134,7 +134,7 @@ public interface EnumOrBuilder extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 4;</code>
+   * <code>.google.protobuf.SourceContext source_context = 4;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder();
 
@@ -143,7 +143,7 @@ public interface EnumOrBuilder extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+   * <code>.google.protobuf.Syntax syntax = 5;</code>
    */
   int getSyntaxValue();
   /**
@@ -151,7 +151,7 @@ public interface EnumOrBuilder extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 5;</code>
+   * <code>.google.protobuf.Syntax syntax = 5;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValue.java
index 83771f5..a0a9d49 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValue.java
@@ -103,7 +103,7 @@ public  final class EnumValue extends
    * Enum value name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -122,7 +122,7 @@ public  final class EnumValue extends
    * Enum value name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -145,7 +145,7 @@ public  final class EnumValue extends
    * Enum value number.
    * </pre>
    *
-   * <code>optional int32 number = 2;</code>
+   * <code>int32 number = 2;</code>
    */
   public int getNumber() {
     return number_;
@@ -276,7 +276,7 @@ public  final class EnumValue extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + NAME_FIELD_NUMBER;
     hash = (53 * hash) + getName().hashCode();
     hash = (37 * hash) + NUMBER_FIELD_NUMBER;
@@ -561,7 +561,7 @@ public  final class EnumValue extends
      * Enum value name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -580,7 +580,7 @@ public  final class EnumValue extends
      * Enum value name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -600,7 +600,7 @@ public  final class EnumValue extends
      * Enum value name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -617,7 +617,7 @@ public  final class EnumValue extends
      * Enum value name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder clearName() {
       
@@ -630,7 +630,7 @@ public  final class EnumValue extends
      * Enum value name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -650,7 +650,7 @@ public  final class EnumValue extends
      * Enum value number.
      * </pre>
      *
-     * <code>optional int32 number = 2;</code>
+     * <code>int32 number = 2;</code>
      */
     public int getNumber() {
       return number_;
@@ -660,7 +660,7 @@ public  final class EnumValue extends
      * Enum value number.
      * </pre>
      *
-     * <code>optional int32 number = 2;</code>
+     * <code>int32 number = 2;</code>
      */
     public Builder setNumber(int value) {
       
@@ -673,7 +673,7 @@ public  final class EnumValue extends
      * Enum value number.
      * </pre>
      *
-     * <code>optional int32 number = 2;</code>
+     * <code>int32 number = 2;</code>
      */
     public Builder clearNumber() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValueOrBuilder.java
index a6b9e7a..22523d7 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/EnumValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface EnumValueOrBuilder extends
    * Enum value name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   java.lang.String getName();
   /**
@@ -20,7 +20,7 @@ public interface EnumValueOrBuilder extends
    * Enum value name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
@@ -30,7 +30,7 @@ public interface EnumValueOrBuilder extends
    * Enum value number.
    * </pre>
    *
-   * <code>optional int32 number = 2;</code>
+   * <code>int32 number = 2;</code>
    */
   int getNumber();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistry.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistry.java
index 3bc5977..c15eafd 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistry.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ExtensionRegistry.java
@@ -32,7 +32,6 @@ package org.apache.hadoop.hbase.shaded.com.google.protobuf;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor;
-
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;


[45/50] [abbrv] hbase git commit: HBASE-17366 Run TestHFile#testReaderWithoutBlockCache failes

Posted by sy...@apache.org.
HBASE-17366 Run TestHFile#testReaderWithoutBlockCache failes

Signed-off-by: CHIA-PING TSAI <ch...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: c1ac3f7739f8c9e20f6aed428558128339467d04
Parents: 363f627
Author: huaxiang sun <hs...@cloudera.com>
Authored: Mon Apr 17 10:32:17 2017 +0800
Committer: CHIA-PING TSAI <ch...@gmail.com>
Committed: Mon Apr 17 10:34:17 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/StoreFileWriter.java   | 9 +++++++++
 .../java/org/apache/hadoop/hbase/io/hfile/TestHFile.java    | 2 +-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c1ac3f77/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index ccfd735..88cba75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -384,6 +384,15 @@ public class StoreFileWriter implements CellSink, ShipperListener {
     }
 
     /**
+     * Creates Builder with cache configuration disabled
+     */
+    public Builder(Configuration conf, FileSystem fs) {
+      this.conf = conf;
+      this.cacheConf = CacheConfig.DISABLED;
+      this.fs = fs;
+    }
+
+    /**
      * @param trt A premade TimeRangeTracker to use rather than build one per append (building one
      * of these is expensive so good to pass one in if you have one).
      * @return this (for chained invocation)

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1ac3f77/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 7074c9d..4db459a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -115,7 +115,7 @@ public class TestHFile  {
     Path storeFileParentDir = new Path(TEST_UTIL.getDataTestDir(), "TestHFile");
     HFileContext meta = new HFileContextBuilder().withBlockSize(64 * 1024).build();
     StoreFileWriter sfw =
-        new StoreFileWriter.Builder(conf, cacheConf, fs).withOutputDir(storeFileParentDir)
+        new StoreFileWriter.Builder(conf, fs).withOutputDir(storeFileParentDir)
             .withComparator(CellComparator.COMPARATOR).withFileContext(meta).build();
 
     final int rowLen = 32;


[30/50] [abbrv] hbase git commit: HBASE-17905: [hbase-spark] bulkload does not work when table not exist

Posted by sy...@apache.org.
HBASE-17905: [hbase-spark] bulkload does not work when table not exist

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: 22f602cab5e9739a650fc962f4b08a0ccc51a972
Parents: 0b5bd78
Author: Yi Liang <ea...@gmail.com>
Authored: Tue Apr 11 15:30:13 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Apr 11 17:01:07 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/spark/BulkLoadPartitioner.scala  | 13 ++++++++-----
 .../apache/hadoop/hbase/spark/HBaseContext.scala  | 18 +++++++++++++++++-
 2 files changed, 25 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/22f602ca/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
index ab4fc41..022c933 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
@@ -33,8 +33,8 @@ import org.apache.spark.Partitioner
 @InterfaceAudience.Public
 class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
   extends Partitioner {
-
-  override def numPartitions: Int = startKeys.length
+  // when table not exist, startKeys = Byte[0][]
+  override def numPartitions: Int = if (startKeys.length == 0) 1 else startKeys.length
 
   override def getPartition(key: Any): Int = {
 
@@ -53,8 +53,11 @@ class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
         case _ =>
           key.asInstanceOf[Array[Byte]]
       }
-    val partition = util.Arrays.binarySearch(startKeys, rowKey, comparator)
-    if (partition < 0) partition * -1 + -2
-    else partition
+    var partition = util.Arrays.binarySearch(startKeys, rowKey, comparator)
+    if (partition < 0)
+      partition = partition * -1 + -2
+    if (partition < 0)
+      partition = 0
+    partition
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/22f602ca/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 e2891db..8c4e0f4 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
@@ -48,7 +48,7 @@ import org.apache.spark.streaming.dstream.DStream
 import java.io._
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod
-import org.apache.hadoop.fs.{Path, FileSystem}
+import org.apache.hadoop.fs.{Path, FileAlreadyExistsException, FileSystem}
 import scala.collection.mutable
 
 /**
@@ -620,9 +620,17 @@ class HBaseContext(@transient sc: SparkContext,
                   compactionExclude: Boolean = false,
                   maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):
   Unit = {
+    val stagingPath = new Path(stagingDir)
+    val fs = stagingPath.getFileSystem(config)
+    if (fs.exists(stagingPath)) {
+      throw new FileAlreadyExistsException("Path " + stagingDir + " already exist")
+    }
     val conn = HBaseConnectionCache.getConnection(config)
     val regionLocator = conn.getRegionLocator(tableName)
     val startKeys = regionLocator.getStartKeys
+    if (startKeys.length == 0) {
+      logInfo("Table " + tableName.toString + " was not found")
+    }
     val defaultCompressionStr = config.get("hfile.compression",
       Compression.Algorithm.NONE.getName)
     val hfileCompression = HFileWriterImpl
@@ -743,9 +751,17 @@ class HBaseContext(@transient sc: SparkContext,
                   compactionExclude: Boolean = false,
                   maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):
   Unit = {
+    val stagingPath = new Path(stagingDir)
+    val fs = stagingPath.getFileSystem(config)
+    if (fs.exists(stagingPath)) {
+      throw new FileAlreadyExistsException("Path " + stagingDir + " already exist")
+    }
     val conn = HBaseConnectionCache.getConnection(config)
     val regionLocator = conn.getRegionLocator(tableName)
     val startKeys = regionLocator.getStartKeys
+    if (startKeys.length == 0) {
+      logInfo("Table " + tableName.toString + " was not found")
+    }
     val defaultCompressionStr = config.get("hfile.compression",
       Compression.Algorithm.NONE.getName)
     val defaultCompression = HFileWriterImpl


[33/50] [abbrv] hbase git commit: HBASE-17896 The FIXED_OVERHEAD of Segment is incorrect

Posted by sy...@apache.org.
HBASE-17896 The FIXED_OVERHEAD of Segment is incorrect


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

Branch: refs/heads/hbase-12439
Commit: 3aadc675b0f02c3c13be625b40d72fbf6a844964
Parents: d7ddc79
Author: CHIA-PING TSAI <ch...@gmail.com>
Authored: Tue Apr 11 16:31:20 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Wed Apr 12 11:26:43 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/util/ClassSize.java |  7 +++++++
 .../apache/hadoop/hbase/regionserver/Segment.java    |  7 ++++---
 .../org/apache/hadoop/hbase/io/TestHeapSize.java     | 15 +++++++++------
 3 files changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3aadc675/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 465bd9c..e1690c0 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
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.hbase.util;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.util.concurrent.ConcurrentHashMap;
@@ -235,6 +236,12 @@ public class ClassSize {
   }
 
   private static final MemoryLayout memoryLayout = getMemoryLayout();
+  private static final boolean USE_UNSAFE_LAYOUT = (memoryLayout instanceof UnsafeLayout);
+
+  @VisibleForTesting
+  public static boolean useUnsafeLayout() {
+    return USE_UNSAFE_LAYOUT;
+  }
 
   /**
    * Method for reading the arc settings and setting overheads according

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aadc675/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
index 6f431c9..8f43fa8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
@@ -48,12 +48,13 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 public abstract class Segment {
 
-  final static long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
-      + 5 * ClassSize.REFERENCE // cellSet, comparator, memStoreLAB, size, timeRangeTracker
+  public final static long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
+      + 6 * ClassSize.REFERENCE // cellSet, comparator, memStoreLAB, dataSize,
+                                // heapSize, and timeRangeTracker
       + Bytes.SIZEOF_LONG // minSequenceId
       + Bytes.SIZEOF_BOOLEAN); // tagsPresent
   public final static long DEEP_OVERHEAD = FIXED_OVERHEAD + ClassSize.ATOMIC_REFERENCE
-      + ClassSize.CELL_SET + ClassSize.ATOMIC_LONG + ClassSize.TIMERANGE_TRACKER;
+      + ClassSize.CELL_SET + 2 * ClassSize.ATOMIC_LONG + ClassSize.TIMERANGE_TRACKER;
 
   private AtomicReference<CellSet> cellSet= new AtomicReference<>();
   private final CellComparator comparator;

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aadc675/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
index 6b943a7..bf74a9e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
@@ -344,7 +344,7 @@ public class TestHeapSize  {
     cl = Segment.class;
     actual = Segment.DEEP_OVERHEAD;
     expected = ClassSize.estimateBase(cl, false);
-    expected += ClassSize.estimateBase(AtomicLong.class, false);
+    expected += 2 * ClassSize.estimateBase(AtomicLong.class, false);
     expected += ClassSize.estimateBase(AtomicReference.class, false);
     expected += ClassSize.estimateBase(CellSet.class, false);
     expected += ClassSize.estimateBase(TimeRangeTracker.class, false);
@@ -361,7 +361,7 @@ public class TestHeapSize  {
     cl = MutableSegment.class;
     actual = MutableSegment.DEEP_OVERHEAD;
     expected = ClassSize.estimateBase(cl, false);
-    expected += ClassSize.estimateBase(AtomicLong.class, false);
+    expected += 2 * ClassSize.estimateBase(AtomicLong.class, false);
     expected += ClassSize.estimateBase(AtomicReference.class, false);
     expected += ClassSize.estimateBase(CellSet.class, false);
     expected += ClassSize.estimateBase(TimeRangeTracker.class, false);
@@ -380,7 +380,7 @@ public class TestHeapSize  {
     cl = ImmutableSegment.class;
     actual = ImmutableSegment.DEEP_OVERHEAD_CSLM;
     expected = ClassSize.estimateBase(cl, false);
-    expected += ClassSize.estimateBase(AtomicLong.class, false);
+    expected += 2 * ClassSize.estimateBase(AtomicLong.class, false);
     expected += ClassSize.estimateBase(AtomicReference.class, false);
     expected += ClassSize.estimateBase(CellSet.class, false);
     expected += ClassSize.estimateBase(TimeRangeTracker.class, false);
@@ -398,7 +398,7 @@ public class TestHeapSize  {
     }
     actual = ImmutableSegment.DEEP_OVERHEAD_CAM;
     expected = ClassSize.estimateBase(cl, false);
-    expected += ClassSize.estimateBase(AtomicLong.class, false);
+    expected += 2 * ClassSize.estimateBase(AtomicLong.class, false);
     expected += ClassSize.estimateBase(AtomicReference.class, false);
     expected += ClassSize.estimateBase(CellSet.class, false);
     expected += ClassSize.estimateBase(TimeRangeTracker.class, false);
@@ -504,8 +504,11 @@ public class TestHeapSize  {
     } else {
       assertTrue(ClassSize.OBJECT == 12 || ClassSize.OBJECT == 16); // depending on CompressedOops
     }
-    assertEquals(ClassSize.OBJECT + 4, ClassSize.ARRAY);
+    if (ClassSize.useUnsafeLayout()) {
+      assertEquals(ClassSize.OBJECT + 4, ClassSize.ARRAY);
+    } else {
+      assertEquals(ClassSize.OBJECT + 8, ClassSize.ARRAY);
+    }
   }
-
 }
 


[20/50] [abbrv] hbase git commit: HBASE-17871 scan#setBatch(int) call leads wrong result of VerifyReplication

Posted by sy...@apache.org.
HBASE-17871 scan#setBatch(int) call leads wrong result of VerifyReplication

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: ec5188df3090d42088b6f4cb8f0c2fd49425f8c1
Parents: d7e3116
Author: Tomu Tsuruhara <to...@linecorp.com>
Authored: Wed Apr 5 21:42:28 2017 +0900
Committer: tedyu <yu...@gmail.com>
Committed: Thu Apr 6 07:00:13 2017 -0700

----------------------------------------------------------------------
 .../mapreduce/replication/VerifyReplication.java | 19 +++++++++++++++----
 1 file changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ec5188df/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index ba5966b..3f8317b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -77,7 +77,7 @@ public class VerifyReplication extends Configured implements Tool {
   private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";
   static long startTime = 0;
   static long endTime = Long.MAX_VALUE;
-  static int batch = Integer.MAX_VALUE;
+  static int batch = -1;
   static int versions = -1;
   static String tableName = null;
   static String families = null;
@@ -110,6 +110,7 @@ public class VerifyReplication extends Configured implements Tool {
     private int sleepMsBeforeReCompare;
     private String delimiter = "";
     private boolean verbose = false;
+    private int batch = -1;
 
     /**
      * Map method that compares every scanned row with the equivalent from
@@ -128,8 +129,11 @@ public class VerifyReplication extends Configured implements Tool {
         sleepMsBeforeReCompare = conf.getInt(NAME +".sleepMsBeforeReCompare", 0);
         delimiter = conf.get(NAME + ".delimiter", "");
         verbose = conf.getBoolean(NAME +".verbose", false);
+        batch = conf.getInt(NAME + ".batch", -1);
         final Scan scan = new Scan();
-        scan.setBatch(batch);
+        if (batch > 0) {
+          scan.setBatch(batch);
+        }
         scan.setCacheBlocks(false);
         scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));
         long startTime = conf.getLong(NAME + ".startTime", 0);
@@ -329,6 +333,7 @@ public class VerifyReplication extends Configured implements Tool {
     conf.setLong(NAME+".endTime", endTime);
     conf.setInt(NAME +".sleepMsBeforeReCompare", sleepMsBeforeReCompare);
     conf.set(NAME + ".delimiter", delimiter);
+    conf.setInt(NAME + ".batch", batch);
     conf.setBoolean(NAME +".verbose", verbose);
     conf.setBoolean(NAME +".includeDeletedCells", includeDeletedCells);
     if (families != null) {
@@ -356,6 +361,10 @@ public class VerifyReplication extends Configured implements Tool {
     Scan scan = new Scan();
     scan.setTimeRange(startTime, endTime);
     scan.setRaw(includeDeletedCells);
+    scan.setCacheBlocks(false);
+    if (batch > 0) {
+      scan.setBatch(batch);
+    }
     if (versions >= 0) {
       scan.setMaxVersions(versions);
       LOG.info("Number of versions set to " + versions);
@@ -503,7 +512,7 @@ public class VerifyReplication extends Configured implements Tool {
   private static void restoreDefaults() {
     startTime = 0;
     endTime = Long.MAX_VALUE;
-    batch = Integer.MAX_VALUE;
+    batch = -1;
     versions = -1;
     tableName = null;
     families = null;
@@ -521,13 +530,15 @@ public class VerifyReplication extends Configured implements Tool {
     }
     System.err.println("Usage: verifyrep [--starttime=X]" +
         " [--endtime=Y] [--families=A] [--row-prefixes=B] [--delimiter=] [--recomparesleep=] " +
-        "[--verbose] <peerid> <tablename>");
+        "[--batch=] [--verbose] <peerid> <tablename>");
     System.err.println();
     System.err.println("Options:");
     System.err.println(" starttime    beginning of the time range");
     System.err.println("              without endtime means from starttime to forever");
     System.err.println(" endtime      end of the time range");
     System.err.println(" versions     number of cell versions to verify");
+    System.err.println(" batch        batch count for scan, " +
+        "note that result row counts will no longer be actual number of rows when you use this option");
     System.err.println(" raw          includes raw scan if given in options");
     System.err.println(" families     comma-separated list of families to copy");
     System.err.println(" row-prefixes comma-separated list of row key prefixes to filter on ");


[19/50] [abbrv] hbase git commit: HBASE-17886 Fix compatibility of ServerSideScanMetrics

Posted by sy...@apache.org.
HBASE-17886 Fix compatibility of ServerSideScanMetrics


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

Branch: refs/heads/hbase-12439
Commit: d7e3116a1744057359ca48d94aa50d7fdf0db974
Parents: 17737b2
Author: Yu Li <li...@apache.org>
Authored: Thu Apr 6 17:29:22 2017 +0800
Committer: Yu Li <li...@apache.org>
Committed: Thu Apr 6 17:29:22 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/metrics/ServerSideScanMetrics.java      | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d7e3116a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
index 8a96aeb..03764ed 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
@@ -49,6 +49,11 @@ public class ServerSideScanMetrics {
   public static final String COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME = "ROWS_SCANNED";
   public static final String COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME = "ROWS_FILTERED";
 
+  /** @deprecated Use {@link #COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME} instead */
+  public static final String COUNT_OF_ROWS_SCANNED_KEY = COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME;
+  /** @deprecated Use {@link #COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME} instead */
+  public static final String COUNT_OF_ROWS_FILTERED_KEY = COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME;
+
   /**
    * number of rows filtered during scan RPC
    */


[21/50] [abbrv] hbase git commit: HBASE-17863: Procedure V2: Some cleanup around Procedure.isFinished() and procedure executor

Posted by sy...@apache.org.
HBASE-17863: Procedure V2: Some cleanup around Procedure.isFinished() and procedure executor

Signed-off-by: Michael 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/91098038
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/91098038
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/91098038

Branch: refs/heads/hbase-12439
Commit: 9109803891e256f8c047af72572f07695e604a3f
Parents: ec5188d
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Mon Apr 3 17:37:41 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Apr 6 12:05:23 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ProcedureState.java |  2 +-
 .../hadoop/hbase/procedure2/Procedure.java      | 48 ++++++++++++--------
 .../hbase/procedure2/ProcedureExecutor.java     | 32 ++++++-------
 .../hadoop/hbase/procedure2/ProcedureUtil.java  |  2 +-
 .../hbase/procedure2/store/ProcedureStore.java  |  2 +-
 .../store/wal/ProcedureWALFormatReader.java     | 19 ++++----
 .../procedure2/ProcedureTestingUtility.java     |  4 +-
 ...ProcedureWALLoaderPerformanceEvaluation.java |  2 +-
 .../protobuf/generated/ProcedureProtos.java     | 40 +++++++++++-----
 .../src/main/protobuf/Procedure.proto           |  3 +-
 .../master/procedure/TestProcedureAdmin.java    |  5 +-
 11 files changed, 92 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
index 5d95add..0080baa 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
@@ -24,5 +24,5 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  */
 @InterfaceAudience.Public
 public enum ProcedureState {
-  INITIALIZING, RUNNABLE, WAITING, WAITING_TIMEOUT, ROLLEDBACK, FINISHED;
+  INITIALIZING, RUNNABLE, WAITING, WAITING_TIMEOUT, ROLLEDBACK, SUCCESS, FAILED;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 2a7fa6e..761ab3a 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -216,9 +216,9 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   }
 
   /**
-   * By default, the executor will try ro run procedures start to finish.
+   * By default, the executor will try to run procedures start to finish.
    * Return true to make the executor yield between each execution step to
-   * give other procedures time to run their steps.
+   * give other procedures a chance to run.
    * @param env the environment passed to the ProcedureExecutor
    * @return Return true if the executor should yield on completion of an execution step.
    *         Defaults to return false.
@@ -271,7 +271,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
     toStringState(sb);
 
     if (hasException()) {
-      sb.append(", failed=" + getException());
+      sb.append(", exception=" + getException());
     }
 
     sb.append(", ");
@@ -506,6 +506,25 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   // ==============================================================================================
 
   /**
+   * Procedure has states which are defined in proto file. At some places in the code, we
+   * need to determine more about those states. Following Methods help determine:
+   *
+   * {@link #isFailed()} - A procedure has executed at least once and has failed. The procedure
+   *                       may or may not have rolled back yet. Any procedure in FAILED state
+   *                       will be eventually moved to ROLLEDBACK state.
+   *
+   * {@link #isSuccess()} - A procedure is completed successfully without any exception.
+   *
+   * {@link #isFinished()} - As a procedure in FAILED state will be tried forever for rollback, only
+   *                         condition when scheduler/ executor will drop procedure from further
+   *                         processing is when procedure state is ROLLEDBACK or isSuccess()
+   *                         returns true. This is a terminal state of the procedure.
+   *
+   * {@link #isWaiting()} - Procedure is in one of the two waiting states ({@link
+   *                        ProcedureState#WAITING}, {@link ProcedureState#WAITING_TIMEOUT}).
+   */
+
+  /**
    * @return true if the procedure is in a RUNNABLE state.
    */
   protected synchronized boolean isRunnable() {
@@ -517,34 +536,25 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   }
 
   /**
-   * @return true if the procedure has failed.
-   *         true may mean failed but not yet rolledback or failed and rolledback.
+   * @return true if the procedure has failed. It may or may not have rolled back.
    */
   public synchronized boolean isFailed() {
-    return exception != null || state == ProcedureState.ROLLEDBACK;
+    return state == ProcedureState.FAILED || state == ProcedureState.ROLLEDBACK;
   }
 
   /**
    * @return true if the procedure is finished successfully.
    */
   public synchronized boolean isSuccess() {
-    return state == ProcedureState.FINISHED && exception == null;
+    return state == ProcedureState.SUCCESS && !hasException();
   }
 
   /**
-   * @return true if the procedure is finished. The Procedure may be completed
-   *         successfuly or failed and rolledback.
+   * @return true if the procedure is finished. The Procedure may be completed successfully or
+   * rolledback.
    */
   public synchronized boolean isFinished() {
-    switch (state) {
-      case ROLLEDBACK:
-        return true;
-      case FINISHED:
-        return exception == null;
-      default:
-        break;
-    }
-    return false;
+    return isSuccess() || state == ProcedureState.ROLLEDBACK;
   }
 
   /**
@@ -580,7 +590,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   protected synchronized void setFailure(final RemoteProcedureException exception) {
     this.exception = exception;
     if (!isFinished()) {
-      setState(ProcedureState.FINISHED);
+      setState(ProcedureState.FAILED);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index e2f63c6..8832637 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -313,7 +313,7 @@ public class ProcedureExecutor<TEnvironment> {
       final NonceKey nonceKey;
       final long procId;
 
-      if (procIter.isNextCompleted()) {
+      if (procIter.isNextFinished()) {
         ProcedureInfo proc = procIter.nextAsProcedureInfo();
         nonceKey = proc.getNonceKey();
         procId = proc.getProcId();
@@ -351,7 +351,7 @@ public class ProcedureExecutor<TEnvironment> {
     HashSet<Procedure> waitingSet = null;
     procIter.reset();
     while (procIter.hasNext()) {
-      if (procIter.isNextCompleted()) {
+      if (procIter.isNextFinished()) {
         procIter.skipNext();
         continue;
       }
@@ -397,11 +397,9 @@ public class ProcedureExecutor<TEnvironment> {
           }
           waitingSet.add(proc);
           break;
-        case FINISHED:
-          if (proc.hasException()) {
-            // add the proc to the scheduler to perform the rollback
-            scheduler.addBack(proc);
-          }
+        case FAILED:
+          // add the proc to the scheduler to perform the rollback
+          scheduler.addBack(proc);
           break;
         case ROLLEDBACK:
         case INITIALIZING:
@@ -650,7 +648,7 @@ public class ProcedureExecutor<TEnvironment> {
    * @return whether the chore is removed, or it will be removed later
    */
   public boolean removeChore(final ProcedureInMemoryChore chore) {
-    chore.setState(ProcedureState.FINISHED);
+    chore.setState(ProcedureState.SUCCESS);
     return timeoutExecutor.remove(chore);
   }
 
@@ -1317,7 +1315,7 @@ public class ProcedureExecutor<TEnvironment> {
     Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE);
 
     // Execute the procedure
-    boolean isSuspended = false;
+    boolean suspended = false;
     boolean reExecute = false;
     Procedure[] subprocs = null;
     do {
@@ -1328,7 +1326,7 @@ public class ProcedureExecutor<TEnvironment> {
           subprocs = null;
         }
       } catch (ProcedureSuspendedException e) {
-        isSuspended = true;
+        suspended = true;
       } catch (ProcedureYieldException e) {
         if (LOG.isTraceEnabled()) {
           LOG.trace("Yield " + procedure + ": " + e.getMessage());
@@ -1358,9 +1356,9 @@ public class ProcedureExecutor<TEnvironment> {
           }
         } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
           timeoutExecutor.add(procedure);
-        } else if (!isSuspended) {
+        } else if (!suspended) {
           // No subtask, so we are done
-          procedure.setState(ProcedureState.FINISHED);
+          procedure.setState(ProcedureState.SUCCESS);
         }
       }
 
@@ -1369,20 +1367,20 @@ public class ProcedureExecutor<TEnvironment> {
 
       // allows to kill the executor before something is stored to the wal.
       // useful to test the procedure recovery.
-      if (testing != null && testing.shouldKillBeforeStoreUpdate(isSuspended)) {
+      if (testing != null && testing.shouldKillBeforeStoreUpdate(suspended)) {
         LOG.debug("TESTING: Kill before store update: " + procedure);
         stop();
         return;
       }
 
-      // Commit the transaction
-      updateStoreOnExec(procStack, procedure, subprocs);
-
       // if the store is not running we are aborting
       if (!store.isRunning()) return;
 
+      // Commit the transaction
+      updateStoreOnExec(procStack, procedure, subprocs);
+
       // if the procedure is kind enough to pass the slot to someone else, yield
-      if (procedure.isRunnable() && !isSuspended &&
+      if (procedure.isRunnable() && !suspended &&
           procedure.isYieldAfterExecutionStep(getEnvironment())) {
         scheduler.yield(procedure);
         return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
index 452a37b..372dcac 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
@@ -184,7 +184,7 @@ public final class ProcedureUtil {
     }
 
     if (proto.hasException()) {
-      assert proc.getState() == ProcedureProtos.ProcedureState.FINISHED ||
+      assert proc.getState() == ProcedureProtos.ProcedureState.FAILED ||
              proc.getState() == ProcedureProtos.ProcedureState.ROLLEDBACK :
              "The procedure must be failed (waiting to rollback) or rolledback";
       proc.setFailure(RemoteProcedureException.fromProto(proto.getException()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
index 032c8fc..385cedb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
@@ -70,7 +70,7 @@ public interface ProcedureStore {
     /**
      * @return true if the iterator next element is a completed procedure.
      */
-    boolean isNextCompleted();
+    boolean isNextFinished();
 
     /**
      * Skip the next procedure

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/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 4cc459b..c672045 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
@@ -49,7 +49,7 @@ public class ProcedureWALFormatReader {
   //   - INIT: Procedure submitted by the user (also known as 'root procedure')
   //   - INSERT: Children added to the procedure <parentId>:[<childId>, ...]
   //   - UPDATE: The specified procedure was updated
-  //   - DELETE: The procedure was removed (completed/rolledback and result TTL expired)
+  //   - DELETE: The procedure was removed (finished/rolledback and result TTL expired)
   //
   // In the WAL we can find multiple times the same procedure as UPDATE or INSERT.
   // We read the WAL from top to bottom, so every time we find an entry of the
@@ -326,15 +326,14 @@ public class ProcedureWALFormatReader {
     public boolean hasParent() { return proto.hasParentId(); }
     public boolean isReady() { return ready; }
 
-    public boolean isCompleted() {
+    public boolean isFinished() {
       if (!hasParent()) {
-        // we only consider 'root' procedures. because for the user 'completed'
-        // means when everything up to the 'root' is complete.
+        // we only consider 'root' procedures. because for the user 'finished'
+        // means when everything up to the 'root' is finished.
         switch (proto.getState()) {
           case ROLLEDBACK:
+          case SUCCESS:
             return true;
-          case FINISHED:
-            return !proto.hasException();
           default:
             break;
         }
@@ -387,8 +386,8 @@ public class ProcedureWALFormatReader {
     }
 
     @Override
-    public boolean isNextCompleted() {
-      return current != null && current.isCompleted();
+    public boolean isNextFinished() {
+      return current != null && current.isFinished();
     }
 
     @Override
@@ -676,8 +675,8 @@ public class ProcedureWALFormatReader {
     private boolean checkReadyToRun(Entry rootEntry) {
       assert !rootEntry.hasParent() : "expected root procedure, got " + rootEntry;
 
-      if (rootEntry.isCompleted()) {
-        // if the root procedure is completed, sub-procedures should be gone
+      if (rootEntry.isFinished()) {
+        // if the root procedure is finished, sub-procedures should be gone
         if (rootEntry.childHead != null) {
           LOG.error("unexpected active children for root-procedure: " + rootEntry);
           for (Entry p = rootEntry.childHead; p != null; p = p.linkNext) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 0240465..1f4244a 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -409,7 +409,7 @@ public class ProcedureTestingUtility {
     }
 
     public void setFinishedState() {
-      setState(ProcedureState.FINISHED);
+      setState(ProcedureState.SUCCESS);
     }
 
     public void setData(final byte[] data) {
@@ -523,7 +523,7 @@ public class ProcedureTestingUtility {
     public void load(ProcedureIterator procIter) throws IOException {
       while (procIter.hasNext()) {
         long procId;
-        if (procIter.isNextCompleted()) {
+        if (procIter.isNextFinished()) {
           ProcedureInfo proc = procIter.nextAsProcedureInfo();
           procId = proc.getProcId();
           LOG.debug("loading completed procId=" + procId + ": " + proc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
index 3d8aa62..75623d5 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java
@@ -82,7 +82,7 @@ public class ProcedureWALLoaderPerformanceEvaluation extends AbstractHBaseTool {
     @Override
     public void load(ProcedureIterator procIter) throws IOException {
       while (procIter.hasNext()) {
-        if (procIter.isNextCompleted()) {
+        if (procIter.isNextFinished()) {
           ProcedureInfo proc = procIter.nextAsProcedureInfo();
         } else {
           Procedure proc = procIter.nextAsProcedure();

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
index b0ba843..56d1e11 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
@@ -61,12 +61,20 @@ public final class ProcedureProtos {
     ROLLEDBACK(5),
     /**
      * <pre>
-     * The procedure execution is completed. may need a rollback if failed.
+     * The procedure execution is completed successfully.
      * </pre>
      *
-     * <code>FINISHED = 6;</code>
+     * <code>SUCCESS = 6;</code>
      */
-    FINISHED(6),
+    SUCCESS(6),
+    /**
+     * <pre>
+     * The procedure execution is failed, may need to rollback
+     * </pre>
+     *
+     * <code>FAILED = 7;</code>
+     */
+    FAILED(7),
     ;
 
     /**
@@ -111,12 +119,20 @@ public final class ProcedureProtos {
     public static final int ROLLEDBACK_VALUE = 5;
     /**
      * <pre>
-     * The procedure execution is completed. may need a rollback if failed.
+     * The procedure execution is completed successfully.
+     * </pre>
+     *
+     * <code>SUCCESS = 6;</code>
+     */
+    public static final int SUCCESS_VALUE = 6;
+    /**
+     * <pre>
+     * The procedure execution is failed, may need to rollback
      * </pre>
      *
-     * <code>FINISHED = 6;</code>
+     * <code>FAILED = 7;</code>
      */
-    public static final int FINISHED_VALUE = 6;
+    public static final int FAILED_VALUE = 7;
 
 
     public final int getNumber() {
@@ -138,7 +154,8 @@ public final class ProcedureProtos {
         case 3: return WAITING;
         case 4: return WAITING_TIMEOUT;
         case 5: return ROLLEDBACK;
-        case 6: return FINISHED;
+        case 6: return SUCCESS;
+        case 7: return FAILED;
         default: return null;
       }
     }
@@ -7752,11 +7769,12 @@ public final class ProcedureProtos {
       "DURE_WAL_INIT\020\002\022\030\n\024PROCEDURE_WAL_INSERT\020" +
       "\003\022\030\n\024PROCEDURE_WAL_UPDATE\020\004\022\030\n\024PROCEDURE" +
       "_WAL_DELETE\020\005\022\031\n\025PROCEDURE_WAL_COMPACT\020\006" +
-      "*p\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014\n\010" +
+      "*{\n\016ProcedureState\022\020\n\014INITIALIZING\020\001\022\014\n\010" +
       "RUNNABLE\020\002\022\013\n\007WAITING\020\003\022\023\n\017WAITING_TIMEO" +
-      "UT\020\004\022\016\n\nROLLEDBACK\020\005\022\014\n\010FINISHED\020\006BL\n1or",
-      "g.apache.hadoop.hbase.shaded.protobuf.ge" +
-      "neratedB\017ProcedureProtosH\001\210\001\001\240\001\001"
+      "UT\020\004\022\016\n\nROLLEDBACK\020\005\022\013\n\007SUCCESS\020\006\022\n\n\006FAI",
+      "LED\020\007BL\n1org.apache.hadoop.hbase.shaded." +
+      "protobuf.generatedB\017ProcedureProtosH\001\210\001\001" +
+      "\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
index 80b16e6..2fed24d 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
@@ -31,7 +31,8 @@ enum ProcedureState {
   WAITING = 3;              // The procedure is waiting on children to be completed
   WAITING_TIMEOUT = 4;      // The procedure is waiting a timout or an external event
   ROLLEDBACK = 5;           // The procedure failed and was rolledback
-  FINISHED = 6;             // The procedure execution is completed. may need a rollback if failed.
+  SUCCESS = 6;              // The procedure execution is completed successfully.
+  FAILED = 7;               // The procedure execution is failed, may need to rollback
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/91098038/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index f21fe0b..e6e90ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -25,7 +25,6 @@ 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.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -212,7 +211,7 @@ public class TestProcedureAdmin {
         assertTrue(procInfo.getProcState() == ProcedureState.RUNNABLE);
         found = true;
       } else {
-        assertTrue(procInfo.getProcState() == ProcedureState.FINISHED);
+        assertTrue(procInfo.getProcState() == ProcedureState.SUCCESS);
       }
     }
     assertTrue(found);
@@ -223,7 +222,7 @@ public class TestProcedureAdmin {
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
     listProcedures = procExec.listProcedures();
     for (ProcedureInfo procInfo: listProcedures) {
-      assertTrue(procInfo.getProcState() == ProcedureState.FINISHED);
+      assertTrue(procInfo.getProcState() == ProcedureState.SUCCESS);
     }
   }
 


[10/50] [abbrv] hbase git commit: HBASE-17857 Remove IS annotations from IA.Public classes

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiHFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiHFileOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiHFileOutputFormat.java
index dc2fc0d..3c90b59 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiHFileOutputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiHFileOutputFormat.java
@@ -16,7 +16,6 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,7 +43,6 @@ import com.google.common.annotations.VisibleForTesting;
  * <p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @VisibleForTesting
 public class MultiHFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable, Cell> {
   private static final Log LOG = LogFactory.getLog(MultiHFileOutputFormat.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormat.java
index 3099c0d..a8e6837 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormat.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Scan;
@@ -55,7 +54,6 @@ import org.apache.hadoop.hbase.client.Scan;
  * </pre>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MultiTableInputFormat extends MultiTableInputFormatBase implements
     Configurable {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
index 25ea047..e18b3aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
@@ -25,7 +25,6 @@ import java.util.List;
 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.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
@@ -54,7 +53,6 @@ import java.util.Iterator;
  * filters etc. Subclasses may use other TableRecordReader implementations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class MultiTableInputFormatBase extends
     InputFormat<ImmutableBytesWritable, Result> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
index 7feb7a9..4cc784f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
@@ -61,7 +60,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable, Mutation> {
   /** Set this to {@link #WAL_OFF} to turn off write-ahead logging (WAL) */
   public static final String WAL_PROPERTY = "hbase.mapreduce.multitableoutputformat.wal";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormat.java
index bd530c8..6ba8138 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormat.java
@@ -22,7 +22,6 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 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.client.Scan;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -78,7 +77,6 @@ import java.util.Map;
  * @see org.apache.hadoop.hbase.client.TableSnapshotScanner
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MultiTableSnapshotInputFormat extends TableSnapshotInputFormat {
 
   private final MultiTableSnapshotInputFormatImpl delegate;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java
index fef275b..8997da9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java
@@ -22,7 +22,6 @@ import java.io.InputStream;
 import java.io.OutputStream;
 
 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.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -34,7 +33,6 @@ import org.apache.hadoop.io.serializer.Serialization;
 import org.apache.hadoop.io.serializer.Serializer;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MutationSerialization implements Serialization<Mutation> {
   @Override
   public boolean accept(Class<?> c) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java
index 1a0cfdb..f01e84f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java
@@ -26,7 +26,6 @@ import java.util.Map;
 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.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -39,7 +38,6 @@ import org.apache.hadoop.mapreduce.Reducer;
  * @see TableMapReduceUtil
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class PutCombiner<K> extends Reducer<K, Put, K, Put> {
   private static final Log LOG = LogFactory.getLog(PutCombiner.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java
index b48580d..17ab9cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -52,7 +51,6 @@ import org.apache.hadoop.util.StringUtils;
  * @see KeyValueSortReducer
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class PutSortReducer extends
     Reducer<ImmutableBytesWritable, Put, ImmutableBytesWritable, KeyValue> {
   // the cell creator

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java
index 98c92ea..dff04b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
@@ -42,7 +41,6 @@ import org.apache.hadoop.io.serializer.Serialization;
 import org.apache.hadoop.io.serializer.Serializer;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ResultSerialization extends Configured implements Serialization<Result> {
   private static final Log LOG = LogFactory.getLog(ResultSerialization.class);
   // The following configuration property indicates import file format version.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
index 46d29eb..2e0591e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -49,7 +48,6 @@ import org.apache.hadoop.util.ToolRunner;
  * input row has columns that have content.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RowCounter extends Configured implements Tool {
 
   private static final Log LOG = LogFactory.getLog(RowCounter.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SimpleTotalOrderPartitioner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SimpleTotalOrderPartitioner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SimpleTotalOrderPartitioner.java
index 2257054..4ba1088 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SimpleTotalOrderPartitioner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SimpleTotalOrderPartitioner.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -45,7 +44,6 @@ import org.apache.hadoop.mapreduce.Partitioner;
  * @see #END
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class SimpleTotalOrderPartitioner<VALUE> extends Partitioner<ImmutableBytesWritable, VALUE>
 implements Configurable {
   private final static Log LOG = LogFactory.getLog(SimpleTotalOrderPartitioner.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
index 4f6b307..63868da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.KeyValue;
 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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -46,7 +45,6 @@ import org.apache.hadoop.util.StringUtils;
  * Convert HBase tabular data into a format that is consumable by Map/Reduce.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableInputFormat extends TableInputFormatBase
 implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index 7962a42..ce1928e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -32,7 +32,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
@@ -97,7 +96,6 @@ import org.apache.hadoop.util.StringUtils;
  * </pre>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class TableInputFormatBase
 extends InputFormat<ImmutableBytesWritable, Result> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index 69b486d..e6a69ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
@@ -69,7 +68,6 @@ import com.codahale.metrics.MetricRegistry;
  */
 @SuppressWarnings({ "rawtypes", "unchecked" })
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableMapReduceUtil {
   private static final Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapper.java
index cde94fe..9a7dcb7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapper.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.mapreduce;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -33,8 +32,7 @@ import org.apache.hadoop.mapreduce.Mapper;
  * @see org.apache.hadoop.mapreduce.Mapper
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class TableMapper<KEYOUT, VALUEOUT>
 extends Mapper<ImmutableBytesWritable, Result, KEYOUT, VALUEOUT> {
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputCommitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputCommitter.java
index 9ccde9a..749fd85 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputCommitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputCommitter.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
@@ -30,7 +29,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * Small committer class that does not do anything.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableOutputCommitter extends OutputCommitter {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 3b5ef8b..615999f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -23,7 +23,6 @@ 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.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -50,7 +49,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * {@link Delete} instance.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableOutputFormat<KEY> extends OutputFormat<KEY, Mutation>
 implements Configurable {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java
index 2b480fe..f66520b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -35,7 +34,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * pairs.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableRecordReader
 extends RecordReader<ImmutableBytesWritable, Result> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
index a8ed5f1..e646727 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -47,7 +46,6 @@ import com.google.common.annotations.VisibleForTesting;
  * pairs.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableRecordReaderImpl {
   public static final String LOG_PER_ROW_COUNT
     = "hbase.mapreduce.log.scanner.rowcount";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableReducer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableReducer.java
index d09311a..f0bfc74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableReducer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableReducer.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.mapreduce;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.mapreduce.Reducer;
 
@@ -41,7 +40,6 @@ import org.apache.hadoop.mapreduce.Reducer;
  * @see org.apache.hadoop.mapreduce.Reducer
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class TableReducer<KEYIN, VALUEIN, KEYOUT>
 extends Reducer<KEYIN, VALUEIN, KEYOUT, Mutation> {
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
index b2db319..15d403f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.fs.Path;
 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;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
@@ -82,7 +81,6 @@ import java.util.List;
  * @see org.apache.hadoop.hbase.client.TableSnapshotScanner
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable, Result> {
 
   public static class TableSnapshotRegionSplit extends InputSplit implements Writable {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
index 1795909..13c7c67 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
@@ -26,7 +26,6 @@ import java.util.Arrays;
 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.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
@@ -40,7 +39,6 @@ import org.apache.hadoop.mapreduce.InputSplit;
  * All references to row below refer to the key of the row.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TableSplit extends InputSplit
 implements Writable, Comparable<TableSplit> {
   /** @deprecated LOG variable would be made private. fix in hbase 3.0 */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
index 05a4820..84324e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.security.visibility.InvalidLabelException;
 import org.apache.hadoop.hbase.util.Base64;
@@ -50,7 +49,6 @@ import org.apache.hadoop.util.StringUtils;
  * @see PutSortReducer
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TextSortReducer extends
     Reducer<ImmutableBytesWritable, Text, ImmutableBytesWritable, KeyValue> {
   

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
index 08b5aab..a9d8e03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
@@ -45,7 +44,6 @@ import org.apache.hadoop.mapreduce.Mapper;
  * Write table content out to files in hdfs.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TsvImporterMapper
 extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
 {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterTextMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterTextMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterTextMapper.java
index 7744ea7..581f0d0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterTextMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterTextMapper.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.IOException;
@@ -34,7 +33,6 @@ import java.io.IOException;
  * Write table content out to map output files.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TsvImporterTextMapper
 extends Mapper<LongWritable, Text, ImmutableBytesWritable, Text>
 {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/VisibilityExpressionResolver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/VisibilityExpressionResolver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/VisibilityExpressionResolver.java
index 48d7708..a83a88f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/VisibilityExpressionResolver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/VisibilityExpressionResolver.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hbase.Tag;
 
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.Tag;
  * Interface to convert visibility expressions into Tags for storing along with Cells in HFiles.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface VisibilityExpressionResolver extends Configurable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/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 d15ffcf..06e43e7 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
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -66,7 +65,6 @@ import org.apache.hadoop.util.ToolRunner;
  * in that case the WAL is replayed for a single table only.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class WALPlayer extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(WALPlayer.class);
   final static String NAME = "WALPlayer";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
index 2892cb2..c76d02a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
@@ -23,14 +23,12 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * The constants used in mob.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class MobConstants {
 
   public static final String MOB_SCAN_RAW = "hbase.mob.scan.raw";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowTooBigException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowTooBigException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowTooBigException.java
index 4a408e7..a4411e4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowTooBigException.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowTooBigException.java
@@ -35,4 +35,4 @@ public class RowTooBigException extends org.apache.hadoop.hbase.client.RowTooBig
   public RowTooBigException(String message) {
     super(message);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index 1d42450..6127d5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -50,7 +49,6 @@ import org.apache.zookeeper.KeeperException;
  * Utility methods for obtaining authentication tokens.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TokenUtil {
   // This class is referenced indirectly by User out in common; instances are created by reflection
   private static final Log LOG = LogFactory.getLog(TokenUtil.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ScanLabelGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ScanLabelGenerator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ScanLabelGenerator.java
index e2425a6..6808545 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ScanLabelGenerator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ScanLabelGenerator.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.security.visibility;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hbase.security.User;
 
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.security.User;
  *
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface ScanLabelGenerator extends Configurable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityExpEvaluator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityExpEvaluator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityExpEvaluator.java
index a720127..8366fce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityExpEvaluator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityExpEvaluator.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.security.visibility;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 
 /**
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.Cell;
  * read results.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface VisibilityExpEvaluator {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelService.java
index 8ddd47e..5c8f0cc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelService.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.OperationStatus;
 import org.apache.hadoop.hbase.security.User;
@@ -34,7 +33,6 @@ import org.apache.hadoop.hbase.security.User;
  * visibility expression storage part and read time evaluation.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface VisibilityLabelService extends Configurable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index efae7e4..e2086e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -36,7 +36,6 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -87,7 +86,6 @@ import org.apache.hadoop.util.Tool;
  * When everything is done, the second cluster can restore the snapshot.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ExportSnapshot extends AbstractHBaseTool implements Tool {
   public static final String NAME = "exportsnapshot";
   /** Configuration prefix for overrides for the source filesystem */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
index 6dbd3f0..eb96438 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
@@ -66,7 +65,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
  * </ol>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class SnapshotInfo extends AbstractHBaseTool {
   private static final Log LOG = LogFactory.getLog(SnapshotInfo.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigurationUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigurationUtil.java
index efb3170..47cb394 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigurationUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConfigurationUtil.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.util;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.util.StringUtils;
 
 import java.util.AbstractMap;
@@ -34,7 +33,6 @@ import java.util.Map;
  * {@link org.apache.hadoop.conf.Configuration} instances.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class ConfigurationUtil {
   // TODO: hopefully this is a good delimiter; it's not in the base64 alphabet, 
   // nor is it valid for paths

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
index 6692ee8..1efab58 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
@@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.io.crypto.DefaultCipherProvider;
@@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class EncryptionTest {
   private static final Log LOG = LogFactory.getLog(EncryptionTest.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LeaseNotRecoveredException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LeaseNotRecoveredException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LeaseNotRecoveredException.java
index ca769b8..152cd8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LeaseNotRecoveredException.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LeaseNotRecoveredException.java
@@ -20,14 +20,12 @@ package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when the lease was expected to be recovered,
  * but the file can't be opened.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class LeaseNotRecoveredException extends HBaseIOException {
   public LeaseNotRecoveredException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
index 7dea269..c6f22ff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
@@ -35,7 +35,6 @@ import java.util.Random;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.zookeeper.server.NIOServerCnxnFactory;
@@ -50,7 +49,6 @@ import com.google.common.annotations.VisibleForTesting;
  * easily access testing helper objects.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MiniZooKeeperCluster {
   private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 696ea18..82c2eab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -61,7 +61,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.Waiter.Predicate;
 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.BufferedMutator;
 import org.apache.hadoop.hbase.client.Connection;
@@ -157,7 +156,6 @@ import edu.umd.cs.findbugs.annotations.Nullable;
  * setting it to true.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @SuppressWarnings("deprecation")
 public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    private MiniZooKeeperCluster zkCluster = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 55529c6..c0efc7b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -26,7 +26,6 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -51,7 +50,6 @@ import org.apache.hadoop.hbase.util.Threads;
  * each and will close down their instance on the way out.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MiniHBaseCluster extends HBaseCluster {
   private static final Log LOG = LogFactory.getLog(MiniHBaseCluster.class.getName());
   public LocalHBaseCluster hbaseCluster;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java
index c1b99f2..6e48ba8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/codec/CodecPerformance.java
@@ -27,7 +27,6 @@ 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.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.KeyValue;
@@ -42,7 +41,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * Do basic codec performance eval.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CodecPerformance {
   /** @deprecated LOG variable would be made private. since 1.2, remove in 3.0 */
   @Deprecated

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
index b11e313..ab4fc41 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
@@ -21,7 +21,6 @@ import java.util
 import java.util.Comparator
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes
 import org.apache.spark.Partitioner
 
@@ -32,7 +31,6 @@ import org.apache.spark.Partitioner
  * @param startKeys   The start keys for the given table
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
   extends Partitioner {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala
index deef6ba..94f63e7 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala
@@ -20,11 +20,9 @@
 package org.apache.hadoop.hbase.spark
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class ByteArrayComparable(val bytes:Array[Byte], val offset:Int = 0, var length:Int = -1)
   extends Comparable[ByteArrayComparable] {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala
index 622da17..8dec2f0 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.spark
 import java.io.Serializable
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes
 
 /**
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.Bytes
  * @param value The Byte Array value
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class ByteArrayWrapper (var value:Array[Byte])
   extends Comparable[ByteArrayWrapper] with Serializable {
   override def compareTo(valueOther: ByteArrayWrapper): Int = {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala
index 8d0885c..3fbb717 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.spark
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes
 
 /**
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.util.Bytes
  * @param qualifierLength    Length of the qualifier value with in the array
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class ColumnFamilyQualifierMapKeyWrapper(val columnFamily:Array[Byte],
                                          val columnFamilyOffSet:Int,
                                          val columnFamilyLength:Int,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala
index e995260..92bb3b7 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.spark
 import java.util
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This object is a clean way to store and sort all cells that will be bulk
  * loaded into a single row
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class FamiliesQualifiersValues extends Serializable {
   //Tree maps are used because we need the results to
   // be sorted when we read them

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala
index 3b517a2..b66889f 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.spark
 import java.io.Serializable
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This object will hold optional data for how a given column family's
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  *                          in the HFile
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class FamilyHFileWriteOptions( val compression:String,
                                val bloomType: String,
                                val blockSize: Int,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/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 3fc0ccc..e2891db 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
@@ -23,7 +23,6 @@ import java.util.UUID
 import javax.management.openmbean.KeyAlreadyExistsException
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.fs.HFileSystem
 import org.apache.hadoop.hbase._
 import org.apache.hadoop.hbase.io.compress.Compression
@@ -61,7 +60,6 @@ import scala.collection.mutable
   * to the working and managing the life cycle of Connections.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class HBaseContext(@transient sc: SparkContext,
                    @transient val config: Configuration,
                    val tmpHdfsConfgFile: String = null)

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala
index be2db02..0d7e7ed 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala
@@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.spark
 
 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._
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable
 import org.apache.spark.streaming.dstream.DStream
@@ -30,7 +29,6 @@ import scala.reflect.ClassTag
  * applied to a Spark DStream so that we can easily interact with HBase
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 object HBaseDStreamFunctions {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala
index bb4439c..ca2d923 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala
@@ -21,7 +21,6 @@ import java.util
 
 import org.apache.hadoop.hbase.{HConstants, TableName}
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client._
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable
 import org.apache.spark.rdd.RDD
@@ -33,7 +32,6 @@ import scala.reflect.ClassTag
  * applied to a Spark RDD so that we can easily interact with HBase
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 object HBaseRDDFunctions
 {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala
index cd5406f..253b386 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.spark
 import org.apache.hadoop.conf.Configuration
 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.{Connection, Delete, Get, Put, Result, Scan}
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable
 import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
@@ -39,7 +38,6 @@ import scala.reflect.ClassTag
  * @param config This is the config information to out HBase cluster
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class JavaHBaseContext(@transient jsc: JavaSparkContext,
                        @transient config: Configuration) extends Serializable {
   val hbaseContext = new HBaseContext(jsc.sc, config)

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala
index cdead5d..722fdae 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.spark
 import java.io.Serializable
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes
 
 /**
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.util.Bytes
  * @param qualifier Cell Qualifier
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class KeyFamilyQualifier(val rowKey:Array[Byte], val family:Array[Byte], val qualifier:Array[Byte])
   extends Comparable[KeyFamilyQualifier] with Serializable {
   override def compareTo(o: KeyFamilyQualifier): Int = {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/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
index f19c590..c695daf 100644
--- 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
@@ -19,13 +19,11 @@ package org.apache.hadoop.hbase.spark
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.InputFormat
 import org.apache.spark.rdd.NewHadoopRDD
 import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext}
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class NewHBaseRDD[K,V](@transient sc : SparkContext,
                        @transient inputFormatClass: Class[_ <: InputFormat[K, V]],
                        @transient keyClass: Class[K],

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
index 8c1cb35..41140ea 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.spark.datasources
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This is the hbase configuration. User can either set them in SparkConf, which
@@ -26,7 +25,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * set in SparkConf. If not set, the default value will take effect.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 object HBaseSparkConf{
   /** Set to false to disable server-side caching of blocks for this scan,
    *  false by default, since full table scans generate too much BC churn.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala b/hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala
index 6515fe9..da79d2a 100644
--- a/hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala
+++ b/hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.datasources.hbase
 
 import org.apache.avro.Schema
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.spark.SchemaConverters
 import org.apache.hadoop.hbase.spark.datasources._
 import org.apache.hadoop.hbase.spark.hbase._
@@ -210,7 +209,6 @@ case class HBaseTableCatalog(
 }
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 object HBaseTableCatalog {
   // If defined and larger than 3, a new table will be created with the nubmer of region specified.
   val newTable = "newtable"


[27/50] [abbrv] hbase git commit: HBASE-17872 The MSLABImpl generates the invaild cells when unsafe is not availble

Posted by sy...@apache.org.
HBASE-17872 The MSLABImpl generates the invaild cells when unsafe is not availble


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

Branch: refs/heads/hbase-12439
Commit: df96d328fb9fa11f04f84607e9a23f254f513202
Parents: 59e8b8e
Author: CHIA-PING TSAI <ch...@gmail.com>
Authored: Sat Apr 8 17:37:37 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Sun Apr 9 23:28:34 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/ByteBufferUtils.java      |  30 ++--
 .../hadoop/hbase/util/TestByteBufferUtils.java  | 165 ++++++++++++++++++-
 .../hbase/util/TestFromClientSide3WoUnsafe.java |  43 +++++
 3 files changed, 213 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/df96d328/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index ff4c843..34a4e02 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -43,15 +43,14 @@ import sun.nio.ch.DirectBuffer;
 @SuppressWarnings("restriction")
 @InterfaceAudience.Public
 public final class ByteBufferUtils {
-
   // "Compressed integer" serialization helper constants.
   public final static int VALUE_MASK = 0x7f;
   public final static int NEXT_BIT_SHIFT = 7;
   public final static int NEXT_BIT_MASK = 1 << 7;
   @VisibleForTesting
-  static boolean UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
+  final static boolean UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
   @VisibleForTesting
-  static boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
+  final static boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
 
   private ByteBufferUtils() {
   }
@@ -404,12 +403,11 @@ public final class ByteBufferUtils {
     } else if (UNSAFE_AVAIL) {
       UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
     } else {
-      int outOldPos = out.position();
-      out.position(destinationOffset);
+      ByteBuffer outDup = out.duplicate();
+      outDup.position(destinationOffset);
       ByteBuffer inDup = in.duplicate();
       inDup.position(sourceOffset).limit(sourceOffset + length);
-      out.put(inDup);
-      out.position(outOldPos);
+      outDup.put(inDup);
     }
     return destinationOffset + length;
   }
@@ -990,7 +988,7 @@ public final class ByteBufferUtils {
 
   /**
    * Copies bytes from given array's offset to length part into the given buffer. Puts the bytes
-   * to buffer's given position.
+   * to buffer's given position. This doesn't affact the position of buffer.
    * @param out
    * @param in
    * @param inOffset
@@ -1003,16 +1001,15 @@ public final class ByteBufferUtils {
     } else if (UNSAFE_AVAIL) {
       UnsafeAccess.copy(in, inOffset, out, outOffset, length);
     } else {
-      int oldPos = out.position();
-      out.position(outOffset);
-      out.put(in, inOffset, length);
-      out.position(oldPos);
+      ByteBuffer outDup = out.duplicate();
+      outDup.position(outOffset);
+      outDup.put(in, inOffset, length);
     }
   }
 
   /**
    * Copies specified number of bytes from given offset of 'in' ByteBuffer to
-   * the array.
+   * the array. This doesn't affact the position of buffer.
    * @param out
    * @param in
    * @param sourceOffset
@@ -1026,10 +1023,9 @@ public final class ByteBufferUtils {
     } else if (UNSAFE_AVAIL) {
       UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
     } else {
-      int oldPos = in.position();
-      in.position(sourceOffset);
-      in.get(out, destinationOffset, length);
-      in.position(oldPos);
+      ByteBuffer inDup = in.duplicate();
+      inDup.position(sourceOffset);
+      inDup.get(out, destinationOffset, length);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/df96d328/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
index 053fb24..ee03c7b 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
@@ -27,14 +27,22 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -50,13 +58,13 @@ import org.junit.runners.Parameterized;
 @Category({MiscTests.class, SmallTests.class})
 @RunWith(Parameterized.class)
 public class TestByteBufferUtils {
-
+  private static final String UNSAFE_AVAIL_NAME = "UNSAFE_AVAIL";
+  private static final String UNSAFE_UNALIGNED_NAME = "UNSAFE_UNALIGNED";
   private byte[] array;
 
   @AfterClass
   public static void afterClass() throws Exception {
-    ByteBufferUtils.UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
-    ByteBufferUtils.UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
+    detectAvailabilityOfUnsafe();
   }
 
   @Parameterized.Parameters
@@ -69,15 +77,50 @@ public class TestByteBufferUtils {
     return paramList;
   }
 
-  public TestByteBufferUtils(boolean useUnsafeIfPossible) {
+  private static void setUnsafe(String fieldName, boolean value) throws Exception {
+    Field field = ByteBufferUtils.class.getDeclaredField(fieldName);
+    field.setAccessible(true);
+    Field modifiersField = Field.class.getDeclaredField("modifiers");
+    modifiersField.setAccessible(true);
+    int oldModifiers = field.getModifiers();
+    modifiersField.setInt(field, oldModifiers & ~Modifier.FINAL);
+    try {
+      field.set(null, value);
+    } finally {
+      modifiersField.setInt(field, oldModifiers);
+    }
+  }
+
+  static void disableUnsafe() throws Exception {
+    if (ByteBufferUtils.UNSAFE_AVAIL) {
+      setUnsafe(UNSAFE_AVAIL_NAME, false);
+    }
+    if (ByteBufferUtils.UNSAFE_UNALIGNED) {
+      setUnsafe(UNSAFE_UNALIGNED_NAME, false);
+    }
+    assertFalse(ByteBufferUtils.UNSAFE_AVAIL);
+    assertFalse(ByteBufferUtils.UNSAFE_UNALIGNED);
+  }
+
+  static void detectAvailabilityOfUnsafe() throws Exception {
+    if (ByteBufferUtils.UNSAFE_AVAIL != UnsafeAvailChecker.isAvailable()) {
+      setUnsafe(UNSAFE_AVAIL_NAME, UnsafeAvailChecker.isAvailable());
+    }
+    if (ByteBufferUtils.UNSAFE_UNALIGNED != UnsafeAvailChecker.unaligned()) {
+      setUnsafe(UNSAFE_UNALIGNED_NAME, UnsafeAvailChecker.unaligned());
+    }
+    assertEquals(ByteBufferUtils.UNSAFE_AVAIL, UnsafeAvailChecker.isAvailable());
+    assertEquals(ByteBufferUtils.UNSAFE_UNALIGNED, UnsafeAvailChecker.unaligned());
+  }
+
+  public TestByteBufferUtils(boolean useUnsafeIfPossible) throws Exception {
     if (useUnsafeIfPossible) {
-      ByteBufferUtils.UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
-      ByteBufferUtils.UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
+      detectAvailabilityOfUnsafe();
     } else {
-      ByteBufferUtils.UNSAFE_AVAIL = false;
-      ByteBufferUtils.UNSAFE_UNALIGNED = false;
+      disableUnsafe();
     }
   }
+
   /**
    * Create an array with sample data.
    */
@@ -388,6 +431,111 @@ public class TestByteBufferUtils {
     assertEquals(i, buffer.getInt());
   }
 
+  private void testCopyFromSrcToDestWithThreads(Object input, Object output,
+    List<Integer> lengthes, List<Integer> offsets) throws InterruptedException {
+    assertTrue((input instanceof ByteBuffer) || (input instanceof byte[]));
+    assertTrue((output instanceof ByteBuffer) || (output instanceof byte[]));
+    assertEquals(lengthes.size(), offsets.size());
+
+    final int threads = lengthes.size();
+    CountDownLatch latch = new CountDownLatch(1);
+    List<Runnable> exes = new ArrayList<>(threads);
+    int oldInputPos = (input instanceof ByteBuffer) ? ((ByteBuffer) input).position() : 0;
+    int oldOutputPos = (output instanceof ByteBuffer) ? ((ByteBuffer) output).position() : 0;
+    for (int i = 0; i != threads; ++i) {
+      int offset = offsets.get(i);
+      int length = lengthes.get(i);
+      exes.add(() -> {
+        try {
+          latch.await();
+          if (input instanceof ByteBuffer && output instanceof byte[]) {
+            ByteBufferUtils.copyFromBufferToArray((byte[]) output,
+                (ByteBuffer) input, offset, offset, length);
+          }
+          if (input instanceof byte[] && output instanceof ByteBuffer) {
+            ByteBufferUtils.copyFromArrayToBuffer((ByteBuffer) output,
+                offset, (byte[]) input, offset, length);
+          }
+          if (input instanceof ByteBuffer && output instanceof ByteBuffer) {
+            ByteBufferUtils.copyFromBufferToBuffer((ByteBuffer) input,
+                (ByteBuffer) output, offset, offset, length);
+          }
+        } catch (InterruptedException ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    ExecutorService service = Executors.newFixedThreadPool(threads);
+    exes.forEach(service::execute);
+    latch.countDown();
+    service.shutdown();
+    assertTrue(service.awaitTermination(5, TimeUnit.SECONDS));
+    if (input instanceof ByteBuffer) {
+      assertEquals(oldInputPos, ((ByteBuffer) input).position());
+    }
+    if (output instanceof ByteBuffer) {
+      assertEquals(oldOutputPos, ((ByteBuffer) output).position());
+    }
+    String inputString = (input instanceof ByteBuffer) ?
+      Bytes.toString(Bytes.toBytes((ByteBuffer) input)) : Bytes.toString((byte[]) input);
+    String outputString = (output instanceof ByteBuffer) ?
+      Bytes.toString(Bytes.toBytes((ByteBuffer) output)) : Bytes.toString((byte[]) output);
+    assertEquals(inputString, outputString);
+  }
+
+  @Test
+  public void testCopyFromSrcToDestWithThreads() throws InterruptedException {
+    List<byte[]> words = Arrays.asList(
+      Bytes.toBytes("with"),
+      Bytes.toBytes("great"),
+      Bytes.toBytes("power"),
+      Bytes.toBytes("comes"),
+      Bytes.toBytes("great"),
+      Bytes.toBytes("responsibility")
+    );
+    List<Integer> lengthes = words.stream().map(v -> v.length).collect(Collectors.toList());
+    List<Integer> offsets = new ArrayList<>(words.size());
+    for (int i = 0; i != words.size(); ++i) {
+      offsets.add(words.subList(0, i).stream().mapToInt(v -> v.length).sum());
+    }
+
+    int totalSize = words.stream().mapToInt(v -> v.length).sum();
+    byte[] fullContent = new byte[totalSize];
+    int offset = 0;
+    for (byte[] w : words) {
+      offset = Bytes.putBytes(fullContent, offset, w, 0, w.length);
+    }
+
+    // test copyFromBufferToArray
+    for (ByteBuffer input : Arrays.asList(
+            ByteBuffer.allocateDirect(totalSize),
+            ByteBuffer.allocate(totalSize))) {
+      words.forEach(input::put);
+      byte[] output = new byte[totalSize];
+      testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets);
+    }
+
+    // test copyFromArrayToBuffer
+    for (ByteBuffer output : Arrays.asList(
+            ByteBuffer.allocateDirect(totalSize),
+            ByteBuffer.allocate(totalSize))) {
+      byte[] input = fullContent;
+      testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets);
+    }
+
+    // test copyFromBufferToBuffer
+    for (ByteBuffer input : Arrays.asList(
+            ByteBuffer.allocateDirect(totalSize),
+            ByteBuffer.allocate(totalSize))) {
+      words.forEach(input::put);
+      for (ByteBuffer output : Arrays.asList(
+            ByteBuffer.allocateDirect(totalSize),
+            ByteBuffer.allocate(totalSize))) {
+        testCopyFromSrcToDestWithThreads(input, output, lengthes, offsets);
+      }
+    }
+  }
+
   @Test
   public void testCopyFromBufferToArray() {
     ByteBuffer buffer = ByteBuffer.allocate(15);
@@ -492,4 +640,5 @@ public class TestByteBufferUtils {
       bb[i] = b;
     }
   }
+
 }

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


[28/50] [abbrv] hbase git commit: HBASE-16477 Remove Writable interface and related code from WALEdit/WALKey

Posted by sy...@apache.org.
HBASE-16477 Remove Writable interface and related code from WALEdit/WALKey


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

Branch: refs/heads/hbase-12439
Commit: 82d554e3783372cc6b05489452c815b57c06f6cd
Parents: df96d32
Author: Enis Soztutar <en...@apache.org>
Authored: Mon Apr 10 02:31:42 2017 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Apr 10 02:31:42 2017 -0700

----------------------------------------------------------------------
 .../regionserver/wal/KeyValueCompression.java   | 133 ------------------
 .../hadoop/hbase/regionserver/wal/WALEdit.java  | 136 +------------------
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |   1 -
 .../org/apache/hadoop/hbase/wal/WALKey.java     |  95 ++-----------
 .../wal/TestKeyValueCompression.java            | 116 ----------------
 5 files changed, 14 insertions(+), 467 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/82d554e3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/KeyValueCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/KeyValueCompression.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/KeyValueCompression.java
deleted file mode 100644
index a33ff9e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/KeyValueCompression.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.regionserver.wal;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * DO NOT USE. This class is deprecated and should only be used in pre-PB WAL.
- * 
- * Compression class for {@link KeyValue}s written to the WAL. This is not
- * synchronized, so synchronization should be handled outside.
- * 
- * Class only compresses and uncompresses row keys, family names, and the
- * qualifier. More may be added depending on use patterns.
- */
-@Deprecated
-@InterfaceAudience.Private
-class KeyValueCompression {
-  /**
-   * Uncompresses a KeyValue from a DataInput and returns it.
-   * 
-   * @param in the DataInput
-   * @param readContext the compressionContext to use.
-   * @return an uncompressed KeyValue
-   * @throws IOException
-   */
-
-  public static KeyValue readKV(DataInput in, CompressionContext readContext)
-      throws IOException {
-    int keylength = WritableUtils.readVInt(in);
-    int vlength = WritableUtils.readVInt(in);
-    int tagsLength = WritableUtils.readVInt(in);
-    int length = (int) KeyValue.getKeyValueDataStructureSize(keylength, vlength, tagsLength);
-
-    byte[] backingArray = new byte[length];
-    int pos = 0;
-    pos = Bytes.putInt(backingArray, pos, keylength);
-    pos = Bytes.putInt(backingArray, pos, vlength);
-
-    // the row
-    int elemLen = Compressor.uncompressIntoArray(backingArray,
-        pos + Bytes.SIZEOF_SHORT, in, readContext.rowDict);
-    checkLength(elemLen, Short.MAX_VALUE);
-    pos = Bytes.putShort(backingArray, pos, (short)elemLen);
-    pos += elemLen;
-
-    // family
-    elemLen = Compressor.uncompressIntoArray(backingArray,
-        pos + Bytes.SIZEOF_BYTE, in, readContext.familyDict);
-    checkLength(elemLen, Byte.MAX_VALUE);
-    pos = Bytes.putByte(backingArray, pos, (byte)elemLen);
-    pos += elemLen;
-
-    // qualifier
-    elemLen = Compressor.uncompressIntoArray(backingArray, pos, in,
-        readContext.qualifierDict);
-    pos += elemLen;
-
-    // the rest
-    in.readFully(backingArray, pos, length - pos);
-
-    return new KeyValue(backingArray, 0, length);
-  }
-
-  private static void checkLength(int len, int max) throws IOException {
-    if (len < 0 || len > max) {
-      throw new IOException(
-          "Invalid length for compresesed portion of keyvalue: " + len);
-    }
-  }
-
-  /**
-   * Compresses and writes ourKV to out, a DataOutput.
-   * 
-   * @param out the DataOutput
-   * @param keyVal the KV to compress and write
-   * @param writeContext the compressionContext to use.
-   * @throws IOException
-   */
-  public static void writeKV(final DataOutput out, KeyValue keyVal,
-      CompressionContext writeContext) throws IOException {
-    byte[] backingArray = keyVal.getBuffer();
-    int offset = keyVal.getOffset();
-
-    // we first write the KeyValue infrastructure as VInts.
-    WritableUtils.writeVInt(out, keyVal.getKeyLength());
-    WritableUtils.writeVInt(out, keyVal.getValueLength());
-    WritableUtils.writeVInt(out, keyVal.getTagsLength());
-
-    // now we write the row key, as the row key is likely to be repeated
-    // We save space only if we attempt to compress elements with duplicates
-    Compressor.writeCompressed(keyVal.getBuffer(), keyVal.getRowOffset(),
-        keyVal.getRowLength(), out, writeContext.rowDict);
-
-  
-    // now family, if it exists. if it doesn't, we write a 0 length array.
-    Compressor.writeCompressed(keyVal.getBuffer(), keyVal.getFamilyOffset(),
-        keyVal.getFamilyLength(), out, writeContext.familyDict);
-
-    // qualifier next
-    Compressor.writeCompressed(keyVal.getBuffer(), keyVal.getQualifierOffset(),
-        keyVal.getQualifierLength(), out,
-        writeContext.qualifierDict);
-
-    // now we write the rest uncompressed
-    int pos = keyVal.getTimestampOffset();
-    int remainingLength = keyVal.getLength() + offset - (pos);
-    out.write(backingArray, pos, remainingLength);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/82d554e3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index 7a8b3d5..d5b95ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -18,22 +18,17 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.NavigableMap;
-import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
@@ -43,7 +38,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.io.Writable;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -51,40 +45,14 @@ import com.google.common.annotations.VisibleForTesting;
 /**
  * WALEdit: Used in HBase's transaction log (WAL) to represent
  * the collection of edits (KeyValue objects) corresponding to a
- * single transaction. The class implements "Writable" interface
- * for serializing/deserializing a set of KeyValue items.
- *
- * Previously, if a transaction contains 3 edits to c1, c2, c3 for a row R,
- * the WAL would have three log entries as follows:
- *
- *    &lt;logseq1-for-edit1&gt;:&lt;eyValue-for-edit-c1&gt;
- *    &lt;logseq2-for-edit2&gt;:&lt;KeyValue-for-edit-c2&gt;
- *    &lt;logseq3-for-edit3&gt;:&lt;KeyValue-for-edit-c3&gt;
- *
- * This presents problems because row level atomicity of transactions
- * was not guaranteed. If we crash after few of the above appends make
- * it, then recovery will restore a partial transaction.
- *
- * In the new world, all the edits for a given transaction are written
- * out as a single record, for example:
- *
- *   &lt;logseq#-for-entire-txn&gt;:&lt;WALEdit-for-entire-txn&gt;
- *
- * where, the WALEdit is serialized as:
- *   &lt;-1, # of edits, &lt;KeyValue&gt;, &lt;KeyValue&gt;, ... &gt;
- * For example:
- *   &lt;-1, 3, &lt;KV-for-edit-c1&gt;, &lt;KV-for-edit-c2&gt;, &lt;KV-for-edit-c3&gt;&gt;
- *
- * The -1 marker is just a special way of being backward compatible with
- * an old WAL which would have contained a single &lt;KeyValue&gt;.
- *
- * The deserializer for WALEdit backward compatibly detects if the record
- * is an old style KeyValue or the new style WALEdit.
+ * single transaction.
  *
+ * All the edits for a given transaction are written out as a single record, in PB format followed
+ * by Cells written via the WALCellEncoder.
  */
 @InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.REPLICATION,
     HBaseInterfaceAudience.COPROC })
-public class WALEdit implements Writable, HeapSize {
+public class WALEdit implements HeapSize {
   private static final Log LOG = LogFactory.getLog(WALEdit.class);
 
   // TODO: Get rid of this; see HBASE-8457
@@ -100,22 +68,10 @@ public class WALEdit implements Writable, HeapSize {
   @VisibleForTesting
   public static final byte [] BULK_LOAD = Bytes.toBytes("HBASE::BULK_LOAD");
 
-  private static final int VERSION_2 = -1;
   private final boolean isReplay;
 
   private ArrayList<Cell> cells = null;
 
-  public static final WALEdit EMPTY_WALEDIT = new WALEdit();
-
-  // Only here for legacy writable deserialization
-  /**
-   * @deprecated Legacy
-   */
-  @Deprecated
-  private NavigableMap<byte[], Integer> scopes;
-
-  private CompressionContext compressionContext;
-
   public WALEdit() {
     this(false);
   }
@@ -162,10 +118,6 @@ public class WALEdit implements Writable, HeapSize {
     return this.isReplay;
   }
 
-  public void setCompressionContext(final CompressionContext compressionContext) {
-    this.compressionContext = compressionContext;
-  }
-
   public WALEdit add(Cell cell) {
     this.cells.add(cell);
     return this;
@@ -195,74 +147,6 @@ public class WALEdit implements Writable, HeapSize {
     this.cells = cells;
   }
 
-  public NavigableMap<byte[], Integer> getAndRemoveScopes() {
-    NavigableMap<byte[], Integer> result = scopes;
-    scopes = null;
-    return result;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    cells.clear();
-    if (scopes != null) {
-      scopes.clear();
-    }
-    int versionOrLength = in.readInt();
-    // TODO: Change version when we protobuf.  Also, change way we serialize KV!  Pb it too.
-    if (versionOrLength == VERSION_2) {
-      // this is new style WAL entry containing multiple KeyValues.
-      int numEdits = in.readInt();
-      for (int idx = 0; idx < numEdits; idx++) {
-        if (compressionContext != null) {
-          this.add(KeyValueCompression.readKV(in, compressionContext));
-        } else {
-          this.add(KeyValueUtil.create(in));
-        }
-      }
-      int numFamilies = in.readInt();
-      if (numFamilies > 0) {
-        if (scopes == null) {
-          scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-        }
-        for (int i = 0; i < numFamilies; i++) {
-          byte[] fam = Bytes.readByteArray(in);
-          int scope = in.readInt();
-          scopes.put(fam, scope);
-        }
-      }
-    } else {
-      // this is an old style WAL entry. The int that we just
-      // read is actually the length of a single KeyValue
-      this.add(KeyValueUtil.create(versionOrLength, in));
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    LOG.warn("WALEdit is being serialized to writable - only expected in test code");
-    out.writeInt(VERSION_2);
-    out.writeInt(cells.size());
-    // We interleave the two lists for code simplicity
-    for (Cell cell : cells) {
-      // This is not used in any of the core code flows so it is just fine to convert to KV
-      KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-      if (compressionContext != null) {
-        KeyValueCompression.writeKV(out, kv, compressionContext);
-      } else{
-        KeyValueUtil.write(kv, out);
-      }
-    }
-    if (scopes == null) {
-      out.writeInt(0);
-    } else {
-      out.writeInt(scopes.size());
-      for (byte[] key : scopes.keySet()) {
-        Bytes.writeByteArray(out, key);
-        out.writeInt(scopes.get(key));
-      }
-    }
-  }
-
   /**
    * Reads WALEdit from cells.
    * @param cellDecoder Cell decoder.
@@ -284,11 +168,6 @@ public class WALEdit implements Writable, HeapSize {
     for (Cell cell : cells) {
       ret += CellUtil.estimatedHeapSizeOf(cell);
     }
-    if (scopes != null) {
-      ret += ClassSize.TREEMAP;
-      ret += ClassSize.align(scopes.size() * ClassSize.MAP_ENTRY);
-      // TODO this isn't quite right, need help here
-    }
     return ret;
   }
 
@@ -301,9 +180,6 @@ public class WALEdit implements Writable, HeapSize {
       sb.append(cell);
       sb.append("; ");
     }
-    if (scopes != null) {
-      sb.append(" scopes: " + scopes.toString());
-    }
     sb.append(">]");
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82d554e3/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 b7adc60..2ae20cf 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
@@ -271,7 +271,6 @@ public interface WAL extends Closeable {
      *          Compression context
      */
     public void setCompressionContext(CompressionContext compressionContext) {
-      edit.setCompressionContext(compressionContext);
       key.setCompressionContext(compressionContext);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/82d554e3/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 9a8003a..bd03e4d 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
@@ -25,7 +25,6 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -59,8 +58,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  *
  * <p>Some Transactional edits (START, COMMIT, ABORT) will not have an associated row.
  *
- * Note that protected members marked @InterfaceAudience.Private are only protected
- * to support the legacy HLogKey class, which is in a different package.
  */
 // TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical
 //       purposes. They need to be merged into WALEntry.
@@ -100,60 +97,9 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     this.sequenceId = writeEntry.getWriteNumber();
   }
 
-  // REMOVE!!!! No more Writables!!!!
-  // Should be < 0 (@see HLogKey#readFields(DataInput))
-  // version 2 supports WAL compression
-  // public members here are only public because of HLogKey
-  @InterfaceAudience.Private
-  protected enum Version {
-    UNVERSIONED(0),
-    // Initial number we put on WALKey when we introduced versioning.
-    INITIAL(-1),
-    // Version -2 introduced a dictionary compression facility.  Only this
-    // dictionary-based compression is available in version -2.
-    COMPRESSED(-2);
-
-    public final int code;
-    static final Version[] byCode;
-    static {
-      byCode = Version.values();
-      for (int i = 0; i < byCode.length; i++) {
-        if (byCode[i].code != -1 * i) {
-          throw new AssertionError("Values in this enum should be descending by one");
-        }
-      }
-    }
-
-    Version(int code) {
-      this.code = code;
-    }
-
-    public boolean atLeast(Version other) {
-      return code <= other.code;
-    }
-
-    public static Version fromCode(int code) {
-      return byCode[code * -1];
-    }
-  }
-
-  /*
-   * This is used for reading the log entries created by the previous releases
-   * (0.94.11) which write the clusters information to the scopes of WALEdit.
-   */
-  private static final String PREFIX_CLUSTER_KEY = ".";
-
+  private byte [] encodedRegionName;
 
-  // visible for deprecated HLogKey
-  @InterfaceAudience.Private
-  protected static final Version VERSION = Version.COMPRESSED;
-
-  // visible for deprecated HLogKey
-  @InterfaceAudience.Private
-  protected byte [] encodedRegionName;
-  // visible for deprecated HLogKey
-  @InterfaceAudience.Private
-  protected TableName tablename;
+  private TableName tablename;
   /**
    * SequenceId for this edit. Set post-construction at write-to-WAL time. Until then it is
    * NO_SEQUENCE_ID. Change it so multiple threads can read it -- e.g. access is synchronized.
@@ -165,15 +111,11 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
    */
   private long origLogSeqNum = 0;
 
-  // Time at which this edit was written.
-  // visible for deprecated HLogKey
-  @InterfaceAudience.Private
-  protected long writeTime;
+  /** Time at which this edit was written. */
+  private long writeTime;
 
-  // The first element in the list is the cluster id on which the change has originated
-  // visible for deprecated HLogKey
-  @InterfaceAudience.Private
-  protected List<UUID> clusterIds;
+  /** The first element in the list is the cluster id on which the change has originated */
+  private List<UUID> clusterIds;
 
   private NavigableMap<byte[], Integer> replicationScope;
 
@@ -186,9 +128,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   private MultiVersionConcurrencyControl.WriteEntry writeEntry;
   public static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
 
-  // visible for deprecated HLogKey
-  @InterfaceAudience.Private
-  protected CompressionContext compressionContext;
+  private CompressionContext compressionContext;
 
   public WALKey() {
     init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
@@ -397,7 +337,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     this.replicationScope = replicationScope;
   }
 
-  // For HLogKey and deserialization. DO NOT USE. See setWriteEntry below.
+  // For deserialization. DO NOT USE. See setWriteEntry below.
   @InterfaceAudience.Private
   protected void setSequenceId(long sequenceId) {
     this.sequenceId = sequenceId;
@@ -486,25 +426,6 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     }
   }
 
-  public void readOlderScopes(NavigableMap<byte[], Integer> scopes) {
-    if (scopes != null) {
-      Iterator<Map.Entry<byte[], Integer>> iterator = scopes.entrySet()
-          .iterator();
-      while (iterator.hasNext()) {
-        Map.Entry<byte[], Integer> scope = iterator.next();
-        String key = Bytes.toString(scope.getKey());
-        if (key.startsWith(PREFIX_CLUSTER_KEY)) {
-          addClusterId(UUID.fromString(key.substring(PREFIX_CLUSTER_KEY
-              .length())));
-          iterator.remove();
-        }
-      }
-      if (scopes.size() > 0) {
-        this.replicationScope = scopes;
-      }
-    }
-  }
-
   /**
    * Marks that the cluster with the given clusterId has consumed the change
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/82d554e3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
deleted file mode 100644
index 4a256a6..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License
- */
-package org.apache.hadoop.hbase.regionserver.wal;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.ArrayBackedTag;
-import org.apache.hadoop.hbase.io.util.LRUDictionary;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.*;
-
-import com.google.common.collect.Lists;
-
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestKeyValueCompression {
-  private static final byte[] VALUE = Bytes.toBytes("fake value");
-  private static final int BUF_SIZE = 256*1024;
-  
-  @Test
-  public void testCountingKVs() throws Exception {
-    List<KeyValue> kvs = Lists.newArrayList();
-    for (int i = 0; i < 400; i++) {
-      byte[] row = Bytes.toBytes("row" + i);
-      byte[] fam = Bytes.toBytes("fam" + i);
-      byte[] qual = Bytes.toBytes("qual" + i);
-      kvs.add(new KeyValue(row, fam, qual, 12345L, VALUE));
-    }
-    
-    runTestCycle(kvs);
-  }
-  
-  @Test
-  public void testRepeatingKVs() throws Exception {
-    List<KeyValue> kvs = Lists.newArrayList();
-    for (int i = 0; i < 400; i++) {
-      byte[] row = Bytes.toBytes("row" + (i % 10));
-      byte[] fam = Bytes.toBytes("fam" + (i % 127));
-      byte[] qual = Bytes.toBytes("qual" + (i % 128));
-      kvs.add(new KeyValue(row, fam, qual, 12345L, VALUE));
-    }
-    
-    runTestCycle(kvs);
-  }
-
-  private void runTestCycle(List<KeyValue> kvs) throws Exception {
-    CompressionContext ctx = new CompressionContext(LRUDictionary.class, false, false);
-    DataOutputBuffer buf = new DataOutputBuffer(BUF_SIZE);
-    for (KeyValue kv : kvs) {
-      KeyValueCompression.writeKV(buf, kv, ctx);
-    }
-
-    ctx.clear();
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(
-        buf.getData(), 0, buf.getLength()));
-    for (KeyValue kv : kvs) {
-      KeyValue readBack = KeyValueCompression.readKV(in, ctx);
-      assertEquals(kv, readBack);
-    }
-  }
-
-  @Test
-  public void testKVWithTags() throws Exception {
-    CompressionContext ctx = new CompressionContext(LRUDictionary.class, false, false);
-    DataOutputBuffer buf = new DataOutputBuffer(BUF_SIZE);
-    KeyValueCompression.writeKV(buf, createKV(1), ctx);
-    KeyValueCompression.writeKV(buf, createKV(0), ctx);
-    KeyValueCompression.writeKV(buf, createKV(2), ctx);
-    
-    ctx.clear();
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(
-        buf.getData(), 0, buf.getLength()));
-    
-    KeyValue readBack = KeyValueCompression.readKV(in, ctx);
-    List<Tag> tags = readBack.getTags();
-    assertEquals(1, tags.size());
-  }
-  
-  private KeyValue createKV(int noOfTags) {
-    byte[] row = Bytes.toBytes("myRow");
-    byte[] cf = Bytes.toBytes("myCF");
-    byte[] q = Bytes.toBytes("myQualifier");
-    byte[] value = Bytes.toBytes("myValue");
-    List<Tag> tags = new ArrayList<>(noOfTags);
-    for (int i = 1; i <= noOfTags; i++) {
-      tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i)));
-    }
-    return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
-  }
-}


[42/50] [abbrv] hbase git commit: HBASE-17866: Implement async setQuota/getQuota methods

Posted by sy...@apache.org.
HBASE-17866: Implement async setQuota/getQuota methods

Signed-off-by: Guanghao Zhang <zg...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 8db9760363890d4d0bfaba25ae6797d45aaf7fec
Parents: 7678855
Author: huzheng <op...@gmail.com>
Authored: Fri Apr 14 14:51:38 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Apr 17 09:49:30 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  16 ++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |  47 +++++
 .../hadoop/hbase/quotas/QuotaRetriever.java     |  32 +--
 .../hadoop/hbase/quotas/QuotaTableUtil.java     |  32 +++
 .../hbase/client/TestAsyncQuotaAdminApi.java    | 207 +++++++++++++++++++
 5 files changed, 306 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index ab791c2..270f28f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.regex.Pattern;
 
@@ -27,6 +28,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.util.Pair;
 
 /**
@@ -465,4 +468,17 @@ public interface AsyncAdmin {
    *          startcode. Here is an example: <code> host187.example.com,60020,1289493121758</code>
    */
   CompletableFuture<Void> move(final byte[] regionName, final byte[] destServerName);
+
+  /**
+   * Apply the new quota settings.
+   * @param quota the quota settings
+   */
+  CompletableFuture<Void> setQuota(final QuotaSettings quota);
+
+  /**
+   * List the quotas based on the filter.
+   * @param filter the quota settings filter
+   * @return the QuotaSetting list, which wrapped by a CompletableFuture.
+   */
+  CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index e42ee57..180cd19 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -56,6 +56,9 @@ import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterReques
 import org.apache.hadoop.hbase.client.Scan.ReadType;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
+import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -112,6 +115,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineReg
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
@@ -1149,6 +1154,48 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
     return future;
   }
 
+  @Override
+  public CompletableFuture<Void> setQuota(QuotaSettings quota){
+    return this.<Void> newMasterCaller()
+        .action((controller, stub) -> this.<SetQuotaRequest, SetQuotaResponse, Void> call(
+          controller, stub, QuotaSettings.buildSetQuotaRequestProto(quota),
+          (s, c, req, done) -> s.setQuota(c, req, done), (resp) -> null))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter) {
+    CompletableFuture<List<QuotaSettings>> future = new CompletableFuture<>();
+    Scan scan = QuotaTableUtil.makeScan(filter);
+    this.connection.getRawTableBuilder(QuotaTableUtil.QUOTA_TABLE_NAME).build().scan(scan,
+      new RawScanResultConsumer() {
+        List<QuotaSettings> settings = new ArrayList<>();
+
+        @Override
+        public void onNext(Result[] results, ScanController controller) {
+          for (Result result : results) {
+            try {
+              QuotaTableUtil.parseResultToCollection(result, settings);
+            } catch (IOException e) {
+              controller.terminate();
+              future.completeExceptionally(e);
+            }
+          }
+        }
+
+        @Override
+        public void onError(Throwable error) {
+          future.completeExceptionally(error);
+        }
+
+        @Override
+        public void onComplete() {
+          future.complete(settings);
+        }
+      });
+    return future;
+  }
+
   private byte[][] getSplitKeys(byte[] startKey, byte[] endKey, int numRegions) {
     if (numRegions < 3) {
       throw new IllegalArgumentException("Must create at least three regions");

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
index cba6a24..0f7baa5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
@@ -86,34 +86,10 @@ public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
   public QuotaSettings next() throws IOException {
     if (cache.isEmpty()) {
       Result result = scanner.next();
-      if (result == null) return null;
-
-      QuotaTableUtil.parseResult(result, new QuotaTableUtil.QuotasVisitor() {
-        @Override
-        public void visitUserQuotas(String userName, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, quotas));
-        }
-
-        @Override
-        public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, table, quotas));
-        }
-
-        @Override
-        public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, namespace, quotas));
-        }
-
-        @Override
-        public void visitTableQuotas(TableName tableName, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromTableQuotas(tableName, quotas));
-        }
-
-        @Override
-        public void visitNamespaceQuotas(String namespace, Quotas quotas) {
-          cache.addAll(QuotaSettingsFactory.fromNamespaceQuotas(namespace, quotas));
-        }
-      });
+      if (result == null) {
+        return null;
+      }
+      QuotaTableUtil.parseResultToCollection(result, cache);
     }
     return cache.poll();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
index 116dd0c..c44090f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.quotas;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.regex.Pattern;
@@ -236,6 +237,37 @@ public class QuotaTableUtil {
     }
   }
 
+  public static void parseResultToCollection(final Result result,
+      Collection<QuotaSettings> quotaSettings) throws IOException {
+
+    QuotaTableUtil.parseResult(result, new QuotaTableUtil.QuotasVisitor() {
+      @Override
+      public void visitUserQuotas(String userName, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, quotas));
+      }
+
+      @Override
+      public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, table, quotas));
+      }
+
+      @Override
+      public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromUserQuotas(userName, namespace, quotas));
+      }
+
+      @Override
+      public void visitTableQuotas(TableName tableName, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromTableQuotas(tableName, quotas));
+      }
+
+      @Override
+      public void visitNamespaceQuotas(String namespace, Quotas quotas) {
+        quotaSettings.addAll(QuotaSettingsFactory.fromNamespaceQuotas(namespace, quotas));
+      }
+    });
+  }
+
   public static void parseNamespaceResult(final Result result,
       final NamespaceQuotasVisitor visitor) throws IOException {
     String namespace = getNamespaceFromRowKey(result.getRow());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8db97603/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java
new file mode 100644
index 0000000..ac9bc16
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.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.client;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.QuotaCache;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
+import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory;
+import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
+import org.apache.hadoop.hbase.quotas.ThrottleType;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+@Category({ ClientTests.class, MediumTests.class })
+public class TestAsyncQuotaAdminApi {
+  private static final Log LOG = LogFactory.getLog(TestAsyncQuotaAdminApi.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static AsyncConnection ASYNC_CONN;
+  protected AsyncAdmin admin;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+    TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, 2000);
+    TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
+    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
+    TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
+    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
+    ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    IOUtils.closeQuietly(ASYNC_CONN);
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    this.admin = ASYNC_CONN.getAdmin();
+  }
+
+  @Test
+  public void testThrottleType() throws Exception {
+    String userName = User.getCurrent().getShortName();
+
+    admin.setQuota(
+      QuotaSettingsFactory.throttleUser(userName, ThrottleType.READ_NUMBER, 6, TimeUnit.MINUTES))
+        .get();
+    admin.setQuota(
+      QuotaSettingsFactory.throttleUser(userName, ThrottleType.WRITE_NUMBER, 12, TimeUnit.MINUTES))
+        .get();
+    admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, true)).get();
+
+    int countThrottle = 0;
+    int countGlobalBypass = 0;
+    for (QuotaSettings settings : admin.getQuota(null).get()) {
+      switch (settings.getQuotaType()) {
+      case THROTTLE:
+        countThrottle++;
+        break;
+      case GLOBAL_BYPASS:
+        countGlobalBypass++;
+        break;
+      default:
+        fail("unexpected settings type: " + settings.getQuotaType());
+      }
+    }
+    assertEquals(2, countThrottle);
+    assertEquals(1, countGlobalBypass);
+
+    admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName)).get();
+    assertNumResults(1, null);
+    admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, false)).get();
+    assertNumResults(0, null);
+  }
+
+  @Test
+  public void testQuotaRetrieverFilter() throws Exception {
+    TableName[] tables = new TableName[] { TableName.valueOf("T0"), TableName.valueOf("T01"),
+        TableName.valueOf("NS0:T2"), };
+    String[] namespaces = new String[] { "NS0", "NS01", "NS2" };
+    String[] users = new String[] { "User0", "User01", "User2" };
+
+    for (String user : users) {
+      admin.setQuota(
+        QuotaSettingsFactory.throttleUser(user, ThrottleType.REQUEST_NUMBER, 1, TimeUnit.MINUTES))
+          .get();
+
+      for (TableName table : tables) {
+        admin.setQuota(QuotaSettingsFactory.throttleUser(user, table, ThrottleType.REQUEST_NUMBER,
+          2, TimeUnit.MINUTES)).get();
+      }
+
+      for (String ns : namespaces) {
+        admin.setQuota(QuotaSettingsFactory.throttleUser(user, ns, ThrottleType.REQUEST_NUMBER, 3,
+          TimeUnit.MINUTES)).get();
+      }
+    }
+    assertNumResults(21, null);
+
+    for (TableName table : tables) {
+      admin.setQuota(
+        QuotaSettingsFactory.throttleTable(table, ThrottleType.REQUEST_NUMBER, 4, TimeUnit.MINUTES))
+          .get();
+    }
+    assertNumResults(24, null);
+
+    for (String ns : namespaces) {
+      admin.setQuota(QuotaSettingsFactory.throttleNamespace(ns, ThrottleType.REQUEST_NUMBER, 5,
+        TimeUnit.MINUTES)).get();
+    }
+    assertNumResults(27, null);
+
+    assertNumResults(7, new QuotaFilter().setUserFilter("User0"));
+    assertNumResults(0, new QuotaFilter().setUserFilter("User"));
+    assertNumResults(21, new QuotaFilter().setUserFilter("User.*"));
+    assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setTableFilter("T0"));
+    assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setTableFilter("NS.*"));
+    assertNumResults(0, new QuotaFilter().setUserFilter("User.*").setTableFilter("T"));
+    assertNumResults(6, new QuotaFilter().setUserFilter("User.*").setTableFilter("T.*"));
+    assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS0"));
+    assertNumResults(0, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS"));
+    assertNumResults(9, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS.*"));
+    assertNumResults(6,
+      new QuotaFilter().setUserFilter("User.*").setTableFilter("T0").setNamespaceFilter("NS0"));
+    assertNumResults(1, new QuotaFilter().setTableFilter("T0"));
+    assertNumResults(0, new QuotaFilter().setTableFilter("T"));
+    assertNumResults(2, new QuotaFilter().setTableFilter("T.*"));
+    assertNumResults(3, new QuotaFilter().setTableFilter(".*T.*"));
+    assertNumResults(1, new QuotaFilter().setNamespaceFilter("NS0"));
+    assertNumResults(0, new QuotaFilter().setNamespaceFilter("NS"));
+    assertNumResults(3, new QuotaFilter().setNamespaceFilter("NS.*"));
+
+    for (String user : users) {
+      admin.setQuota(QuotaSettingsFactory.unthrottleUser(user)).get();
+      for (TableName table : tables) {
+        admin.setQuota(QuotaSettingsFactory.unthrottleUser(user, table)).get();
+      }
+      for (String ns : namespaces) {
+        admin.setQuota(QuotaSettingsFactory.unthrottleUser(user, ns)).get();
+      }
+    }
+    assertNumResults(6, null);
+
+    for (TableName table : tables) {
+      admin.setQuota(QuotaSettingsFactory.unthrottleTable(table)).get();
+    }
+    assertNumResults(3, null);
+
+    for (String ns : namespaces) {
+      admin.setQuota(QuotaSettingsFactory.unthrottleNamespace(ns)).get();
+    }
+    assertNumResults(0, null);
+  }
+
+  private void assertNumResults(int expected, final QuotaFilter filter) throws Exception {
+    assertEquals(expected, countResults(filter));
+  }
+
+  private int countResults(final QuotaFilter filter) throws Exception {
+    int count = 0;
+    for (QuotaSettings settings : admin.getQuota(filter).get()) {
+      LOG.debug(settings);
+      count++;
+    }
+    return count;
+  }
+}


[14/50] [abbrv] hbase git commit: HBASE-17857 Remove IS annotations from IA.Public classes

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index eb1cbc5..179a566 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
@@ -49,7 +48,6 @@ import org.apache.hadoop.hbase.util.ClassSize;
  * {@link #addColumn(byte[], byte[], long)} method.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Increment extends Mutation implements Comparable<Row> {
   private static final long HEAP_OVERHEAD =  ClassSize.REFERENCE + ClassSize.TIMERANGE;
   private TimeRange tr = new TimeRange();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java
index 01aba6f..ad0897e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java
@@ -21,7 +21,6 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Specify Isolation levels in Scan operations.
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * not have been committed yet.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public enum IsolationLevel {
 
   READ_COMMITTED(1),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterSwitchType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterSwitchType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterSwitchType.java
index 7e31b25..5fa9ec2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterSwitchType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterSwitchType.java
@@ -17,13 +17,11 @@
  */
 package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 /**
  * Represents the master switch type
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum MasterSwitchType {
   SPLIT,
   MERGE
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MobCompactPartitionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MobCompactPartitionPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MobCompactPartitionPolicy.java
index f550572..076ab6f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MobCompactPartitionPolicy.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MobCompactPartitionPolicy.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Enum describing the mob compact partition policy types.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum MobCompactPartitionPolicy {
   /**
    * Compact daily mob files into one file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index fb55fdd..b010c2f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagUtil;
 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.io.HeapSize;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -58,7 +57,6 @@ import com.google.common.io.ByteArrayDataOutput;
 import com.google.common.io.ByteStreams;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class Mutation extends OperationWithAttributes implements Row, CellScannable,
     HeapSize {
   public static final long MUTATION_OVERHEAD = ClassSize.align(

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoServerForRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoServerForRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoServerForRegionException.java
index 126b117..e628911 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoServerForRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoServerForRegionException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when no region server can be found for a region
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class NoServerForRegionException extends DoNotRetryRegionException {
   private static final long serialVersionUID = 1L << 11 - 1L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Operation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Operation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Operation.java
index 4f25e2c..130cf16 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Operation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Operation.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.JsonMapper;
 
 /**
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.JsonMapper;
  * Contains methods for exposure to logging and debugging tools.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class Operation {
   // TODO make this configurable
   // TODO Do we need this anymore now we have protobuffed it all?

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
index cc863b9..ba21cbb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
@@ -24,12 +24,10 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class OperationWithAttributes extends Operation implements Attributes {
   // An opaque blob of attributes
   private Map<String, byte[]> attributes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
index 701dceb..5c3ac4b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.IndividualBytesFieldCell;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
@@ -51,7 +50,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * setting the timestamp.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Put extends Mutation implements HeapSize, Comparable<Row> {
   /**
    * Create a Put operation for the specified row.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
index 1322ef5..7f50d13 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
@@ -22,7 +22,6 @@ import java.util.Map;
 
 import com.google.common.collect.Maps;
 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.filter.Filter;
 import org.apache.hadoop.hbase.io.TimeRange;
@@ -38,7 +37,6 @@ import com.google.common.collect.ListMultimap;
 import org.apache.hadoop.hbase.util.Bytes;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class Query extends OperationWithAttributes {
   private static final String ISOLATION_LEVEL = "_isolationlevel_";
   protected Filter filter = null;
@@ -275,4 +273,4 @@ public abstract class Query extends OperationWithAttributes {
   public Map<byte[], TimeRange> getColumnFamilyTimeRange() {
     return this.colFamTimeRangeMap;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
index e493123..4a916d3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
@@ -26,7 +26,6 @@ import java.util.function.Function;
 
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A low level asynchronous table.
@@ -47,7 +46,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * most features of AsyncTable, we can think about merge these two interfaces.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface RawAsyncTable extends AsyncTableBase {
 
   /**
@@ -93,7 +91,6 @@ public interface RawAsyncTable extends AsyncTableBase {
    * </pre>
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Unstable
   @FunctionalInterface
   interface CoprocessorCallable<S, R> {
 
@@ -175,7 +172,6 @@ public interface RawAsyncTable extends AsyncTableBase {
    * </pre>
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Unstable
   interface CoprocessorCallback<R> {
 
     /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
index 899c0bb..820960b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawScanResultConsumer.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 
@@ -32,14 +31,12 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
  * consuming tasks in all methods below unless you know what you are doing.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface RawScanResultConsumer {
 
   /**
    * Used to resume a scan.
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Unstable
   interface ScanResumer {
 
     /**
@@ -60,7 +57,6 @@ public interface RawScanResultConsumer {
    * thrown if you have already called one of the methods.
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Unstable
   interface ScanController {
 
     /**
@@ -122,4 +118,4 @@ public interface RawScanResultConsumer {
    */
   default void onScanMetricsCreated(ScanMetrics scanMetrics) {
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
index bfdb216..5f9cd6d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * POJO representing region server load
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionLoadStats {
   int memstoreLoad;
   int heapOccupancy;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocator.java
index 39518a6..2c96a4a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocator.java
@@ -25,7 +25,6 @@ import java.util.List;
 import org.apache.hadoop.hbase.HRegionLocation;
 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.util.Pair;
 
 /**
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.util.Pair;
  * @since 0.99.0
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface RegionLocator extends Closeable {
   /**
    * Finds the region on which the given row is being served. Does not reload the cache.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionOfflineException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionOfflineException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionOfflineException.java
index d6cceb9..018bc69 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionOfflineException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionOfflineException.java
@@ -20,11 +20,9 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.RegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /** Thrown when a table can not be located */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionOfflineException extends RegionException {
   private static final long serialVersionUID = 466008402L;
   /** default constructor */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java
index 46e730e..33fed2c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java
@@ -25,17 +25,14 @@ import java.util.function.Consumer;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * An interface for client request scheduling algorithm.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface RequestController {
 
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public enum ReturnCode {
     /**
      * Accept current row.
@@ -55,7 +52,6 @@ public interface RequestController {
    * Picks up the valid data.
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public interface Checker {
     /**
      * Checks the data whether it is valid to submit.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java
index 7ed80f0..ba4babd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java
@@ -21,14 +21,12 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 
 /**
  * A factory class that constructs an {@link org.apache.hadoop.hbase.client.RequestController}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class RequestControllerFactory {
   public static final String REQUEST_CONTROLLER_IMPL_CONF_KEY = "hbase.client.request.controller.impl";
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index f8682ec..63aab80 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -79,7 +78,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * in then use {@link #copyFrom(Result)}
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Result implements CellScannable, CellScanner {
   private Cell[] cells;
   private Boolean exists; // if the query was just to check existence.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java
index 8951e84..ef8d887 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java
@@ -26,14 +26,12 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 
 /**
  * Interface for client-side scanning. Go to {@link Table} to obtain instances.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public interface ResultScanner extends Closeable, Iterable<Result> {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java
index dc25f64..eec9f62 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java
@@ -23,14 +23,12 @@ import java.util.Date;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Exception thrown by HTable methods when an attempt to do something (like
  * commit changes) fails after a bunch of retries.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RetriesExhaustedException extends IOException {
   private static final long serialVersionUID = 1876775844L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
index 8b09222..70d5548 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
@@ -30,7 +30,6 @@ import java.util.Set;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RetriesExhaustedWithDetailsException
 extends RetriesExhaustedException {
   List<Throwable> exceptions;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java
index cea45fc..79f27bc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Row.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Has a row.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public interface Row extends Comparable<Row> {
   /**
    * @return The row.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
index 85fd590..758bce6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Provide a way to access the inner buffer.
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * @param <T>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface RowAccess<T> extends Iterable<T> {
   /**
    * @return true if there are no elements.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
index 8a8193e..a9384ac 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * into Sets or using them as keys in Maps.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RowMutations implements Row {
   private final List<Mutation> mutations;
   private final byte [] row;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowTooBigException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowTooBigException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowTooBigException.java
index 69b57b0..e32127c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowTooBigException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowTooBigException.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Gets or Scans throw this exception if running without in-row scan flag
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * hbase.table.max.rowsize).
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RowTooBigException extends DoNotRetryRegionException {
 
   public RowTooBigException(String message) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
index 2b2e4c8..67d1b4d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
@@ -18,12 +18,10 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.IOException;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface RpcRetryingCaller<T> {
   void cancel();
 
@@ -49,4 +47,4 @@ public interface RpcRetryingCaller<T> {
    */
   T callWithoutRetries(RetryingCallable<T> callable, int callTimeout)
   throws IOException, RuntimeException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
index 0047d2f..7bc78d4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
@@ -33,7 +33,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
@@ -87,7 +86,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * instance per usage.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Scan extends Query {
   private static final Log LOG = LogFactory.getLog(Scan.class);
 
@@ -1131,7 +1129,6 @@ public class Scan extends Query {
   }
 
   @InterfaceAudience.Public
-  @InterfaceStability.Unstable
   public enum ReadType {
     DEFAULT, STREAM, PREAD
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java
index 03b1ba0..5a4170f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java
@@ -18,14 +18,12 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 
 /**
  * Receives {@link Result} for an asynchronous scan.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface ScanResultConsumer {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index d70c76f..72b2a15 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Updat
  * networking, etc..) when talking to a local master
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
 
   private final MasterService.BlockingInterface stub;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java
index b9b6b6c..9a1e1cb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java
@@ -19,13 +19,11 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * The POJO equivalent of HBaseProtos.SnapshotDescription
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class SnapshotDescription {
   private final String name;
   private final TableName table;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotType.java
index e3e12bd..c9820ca 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotType.java
@@ -17,13 +17,11 @@
  */
 package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * POJO representing the snapshot type
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum SnapshotType {
   DISABLED, FLUSH, SKIPFLUSH;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java
index fa4e5f1..b74823c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SyncCoprocessorRpcChannel.java
@@ -28,7 +28,6 @@ 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.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 
@@ -39,7 +38,6 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
  * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 abstract class SyncCoprocessorRpcChannel implements CoprocessorRpcChannel {
   private static final Log LOG = LogFactory.getLog(SyncCoprocessorRpcChannel.class);
 
@@ -77,4 +75,4 @@ abstract class SyncCoprocessorRpcChannel implements CoprocessorRpcChannel {
   protected abstract Message callExecService(RpcController controller,
       Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
           throws IOException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index 90fee8d..933329f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 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.coprocessor.Batch;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -49,7 +48,6 @@ import com.google.protobuf.ServiceException;
  * @since 0.99.0
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface Table extends Closeable {
   /**
    * Gets the fully qualified table name instance of this table.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
index 27e1596..3eedb10 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * For creating {@link Table} instance.
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * Table instance.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface TableBuilder {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
index 33aef79..742acee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
@@ -21,10 +21,8 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 class UnmodifyableHRegionInfo extends HRegionInfo {
   /*
    * Creates an unmodifyable copy of an HRegionInfo

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
index 59a1bd5..b5f5ae9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
@@ -22,13 +22,11 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Read-only table descriptor.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class UnmodifyableHTableDescriptor extends HTableDescriptor {
   /**
    * Default constructor.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/WrongRowIOException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/WrongRowIOException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/WrongRowIOException.java
index e0609da..69729f5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/WrongRowIOException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/WrongRowIOException.java
@@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class WrongRowIOException extends HBaseIOException {
   private static final long serialVersionUID = -5849522209440123059L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicy.java
index 94e434f..2c7b139 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicy.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicy.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client.backoff;
 
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Configurable policy for the amount of time a client should wait for a new request to the
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface ClientBackoffPolicy {
 
   public static final String BACKOFF_POLICY_CLASS =
@@ -39,4 +37,4 @@ public interface ClientBackoffPolicy {
    * @return the number of ms to wait on the client based on the
    */
   public long getBackoffTime(ServerName serverName, byte[] region, ServerStatistics stats);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java
index b41133a..a39bd96 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import com.google.common.base.Preconditions;
 
@@ -32,7 +31,6 @@ import com.google.common.base.Preconditions;
  * max backoff to generate the backoff time.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public class ExponentialClientBackoffPolicy implements ClientBackoffPolicy {
 
   private static final Log LOG = LogFactory.getLog(ExponentialClientBackoffPolicy.class);
@@ -104,4 +102,4 @@ public class ExponentialClientBackoffPolicy implements ClientBackoffPolicy {
         "Value %s must be within the range [%s,%s]", valueIn, baseMin, baseMax);
     return ((limitMax - limitMin) * (valueIn - baseMin) / (baseMax - baseMin)) + limitMin;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
index a9c23cc..0b783d7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
@@ -22,14 +22,12 @@ package org.apache.hadoop.hbase.client.coprocessor;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A collection of interfaces and utilities used for interacting with custom RPC
  * interfaces exposed by Coprocessors.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class Batch {
   /**
    * Defines a unit of work to be executed.
@@ -50,7 +48,6 @@ public abstract class Batch {
    * @param <R> the return type from {@link Batch.Call#call(Object)}
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public interface Call<T,R> {
     R call(T instance) throws IOException;
   }
@@ -72,7 +69,6 @@ public abstract class Batch {
    * org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public interface Callback<R> {
     void update(byte[] region, byte[] row, R result);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ScanMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ScanMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ScanMetrics.java
index 73b3892..7813527 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ScanMetrics.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ScanMetrics.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client.metrics;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 
 /**
@@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * for now.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ScanMetrics extends ServerSideScanMetrics {
 
   // AtomicLongs to hold the metrics values. These are all updated through ClientScanner and

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
index b14938b..8a96aeb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/metrics/ServerSideScanMetrics.java
@@ -22,7 +22,6 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import com.google.common.collect.ImmutableMap;
 
@@ -30,7 +29,6 @@ import com.google.common.collect.ImmutableMap;
  * Provides server side metrics related to scan operations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ServerSideScanMetrics {
   /**
    * Hash to hold the String -&gt; Atomic Long mappings for each metric

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index 0eae10b..94425f9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 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;
@@ -86,7 +85,6 @@ import com.google.common.collect.Lists;
  * @deprecated use {@link org.apache.hadoop.hbase.client.Admin} instead.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @Deprecated
 public class ReplicationAdmin implements Closeable {
   private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFs.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFs.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFs.java
index f293586..854517e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFs.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFs.java
@@ -23,14 +23,12 @@ import java.util.Map;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Used by {@link org.apache.hadoop.hbase.client.Admin#listReplicatedTableCFs()}.
  * The cfs is a map of <ColumnFamily, ReplicationScope>.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TableCFs {
   private final TableName table;
   private final Map<String, Integer> cfs;
@@ -58,4 +56,4 @@ public class TableCFs {
     }
     return sb.toString();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/security/SecurityCapability.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/security/SecurityCapability.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/security/SecurityCapability.java
index 1847b2e..ab70616 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/security/SecurityCapability.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/security/SecurityCapability.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.client.security;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Available security capabilities
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum SecurityCapability {
   // Note to implementors: These must match the numbering of Capability values in MasterProtos
   SIMPLE_AUTHENTICATION(0),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java
index 3b01a9e..8c0d054 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.coprocessor;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if a coprocessor rules we should bypass an operation
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class BypassCoprocessorException extends CoprocessorException {
   private static final long serialVersionUID = 5943889011582357043L;
 
@@ -41,4 +39,4 @@ public class BypassCoprocessorException extends CoprocessorException {
   public BypassCoprocessorException(String s) {
     super(s);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorException.java
index 9946d97..541392c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase.coprocessor;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if a coprocessor encounters any exception.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CoprocessorException extends DoNotRetryIOException {
   private static final long serialVersionUID = 4357922136679804887L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java
index 43a4ee4..74621ab 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java
@@ -39,7 +39,6 @@ package org.apache.hadoop.hbase.exceptions;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
 * Thrown when the client believes that we are trying to communicate to has
@@ -49,7 +48,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 * retries and fast fail the operation.
 */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ConnectionClosingException extends IOException {
   public ConnectionClosingException(String string) {
     super(string);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/FailedSanityCheckException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/FailedSanityCheckException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/FailedSanityCheckException.java
index 5bfd2f3..050b2c1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/FailedSanityCheckException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/FailedSanityCheckException.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.exceptions;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Exception thrown if a mutation fails sanity checks.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class FailedSanityCheckException extends org.apache.hadoop.hbase.DoNotRetryIOException {
 
   private static final long serialVersionUID = 1788783640409186240L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MergeRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MergeRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MergeRegionException.java
index b87e400..2291053 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MergeRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MergeRegionException.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.exceptions;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
 
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
  * Thrown when something is wrong in trying to merge two regions.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class MergeRegionException extends DoNotRetryRegionException {
 
   private static final long serialVersionUID = 4970899110066124122L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java
index b31e055..63f8929 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java
@@ -23,7 +23,6 @@ import java.net.ConnectException;
 
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when the client believes that we are trying to communicate to has
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * retries and fast fail the operation.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class PreemptiveFastFailException extends ConnectException {
   private static final long serialVersionUID = 7129103682617007177L;
   private long failureCount, timeOfFirstFailureMilliSec, timeOfLatestAttemptMilliSec;
@@ -107,4 +105,4 @@ public class PreemptiveFastFailException extends ConnectException {
   public boolean isGuaranteedClientSideOnly() {
     return guaranteedClientSideOnly;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java
index 06db472..78ea099 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase.exceptions;
 
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a read request issued against a region which is in recovering state.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionInRecoveryException extends NotServingRegionException {
   private static final long serialVersionUID = 327302071153799L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
index 0021f4a..c71bc6d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.exceptions;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when the size of the rpc request received by the server is too large.
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * @since 1.3.0
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RequestTooBigException extends DoNotRetryIOException {
   private static final long serialVersionUID = -1593339239809586516L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ScannerResetException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ScannerResetException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ScannerResetException.java
index 7689eb1..0704189 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ScannerResetException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ScannerResetException.java
@@ -20,14 +20,12 @@ package org.apache.hadoop.hbase.exceptions;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when the server side has received an Exception, and asks the Client to reset the scanner
  * state by closing the current region scanner, and reopening from the start of last seen row.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ScannerResetException extends DoNotRetryIOException {
   private static final long serialVersionUID = -5649728171144849619L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java
index 933e888..9fbc67d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java
@@ -20,14 +20,12 @@
 package org.apache.hadoop.hbase.exceptions;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * An error requesting an RPC protocol that the server is not serving.
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class UnknownProtocolException extends org.apache.hadoop.hbase.DoNotRetryIOException {
   private Class<?> protocol;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
index b59398b..87b622c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
@@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.filter;
 import java.nio.ByteBuffer;
 
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
@@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class BinaryComparator extends org.apache.hadoop.hbase.filter.ByteArrayComparable {
   /**
    * Constructor

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
index 01cb769..2c951f6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
@@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.filter;
 import java.nio.ByteBuffer;
 
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
@@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * {@link BinaryComparator}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class BinaryPrefixComparator extends ByteArrayComparable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
index dac8864..96ef2e1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
@@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.filter;
 import java.nio.ByteBuffer;
 
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
@@ -34,12 +33,10 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * with the specified byte array. Then returns whether the result is non-zero.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class BitComparator extends ByteArrayComparable {
 
   /** Bit operators. */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public enum BitwiseOp {
     /** and */
     AND,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
index 3ae20a1..dad4132 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.generated.FilterProtos;
 
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * makes this filter unsuitable as a Scan filter.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ColumnCountGetFilter extends FilterBase {
   private int limit = 0;
   private int count = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
index 7d4571e..696f868 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -41,7 +40,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
  * for pagination.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ColumnPaginationFilter extends FilterBase {
 
   private int limit = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
index 7230d3a..b6e9607 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
@@ -42,7 +41,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
  * columns like 'and', 'anti' but not keys with columns like 'ball', 'act'.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ColumnPrefixFilter extends FilterBase {
   protected byte [] prefix = null;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
index 99f9926..69b5088 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@@ -50,7 +49,6 @@ import com.google.common.base.Preconditions;
  * or not.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ColumnRangeFilter extends FilterBase {
   protected byte[] minColumn = null;
   protected boolean minColumnInclusive = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
index bbc31ec..451d7ee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@@ -49,12 +48,10 @@ import com.google.common.base.Preconditions;
  * Multiple filters can be combined using {@link FilterList}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class CompareFilter extends FilterBase {
 
   /** Comparison operators. */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public enum CompareOp {
     /** less than */
     LESS,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
index d82eaec..c14314c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
@@ -28,7 +28,6 @@ import java.util.Set;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -47,7 +46,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
  * full rows for correct filtering 
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class DependentColumnFilter extends CompareFilter {
 
   protected byte[] columnFamily;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
index 8dfd2ca..a28855d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * directly rather than a filter.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class FamilyFilter extends CompareFilter {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
index 59aa855..de5d6c6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
@@ -24,7 +24,6 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 
 /**
@@ -53,7 +52,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
  * @see FilterBase
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class Filter {
   protected transient boolean reversed;
   /**
@@ -156,7 +154,6 @@ public abstract class Filter {
    * Return codes for filterValue().
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public enum ReturnCode {
     /**
      * Include the Cell

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
index 04eba0c..d533026 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -52,11 +51,9 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * Defaults to {@link Operator#MUST_PASS_ALL}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 final public class FilterList extends FilterBase {
   /** set operator */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public static enum Operator {
     /** !AND */
     MUST_PASS_ALL,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
index 14d23d4..8493610 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.generated.FilterProtos;
 
@@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * This filter can be used to more efficiently perform row count operations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class FirstKeyOnlyFilter extends FilterBase {
   private boolean foundKV = false;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
index 4681fd3..ac5f125 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
@@ -24,7 +24,6 @@ import java.util.TreeSet;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -46,7 +45,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
  * @deprecated Deprecated in 2.0. See HBASE-13347
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 @Deprecated
 public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
index 65c2a61..895ffc8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@@ -59,7 +58,6 @@ import com.google.common.annotations.VisibleForTesting;
  * mask is "????_99_????_01", where at ? can be any value.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FuzzyRowFilter extends FilterBase {
   private static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
   private List<Pair<byte[], byte[]>> fuzzyKeysData;


[40/50] [abbrv] hbase git commit: Revert "HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)" Mistaken commit.

Posted by sy...@apache.org.
Revert "HBASE-17906 When a huge amount of data writing to hbase through thrift2, there will be a deadlock error. (Albert Lee)"
Mistaken commit.

This reverts commit 9dd5cda01747ffb91ac084792fa4a8670859e810.


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

Branch: refs/heads/hbase-12439
Commit: 0cd4cec5d24b5e7194a903e4d900f5558ed8b9a7
Parents: c846145
Author: Michael Stack <st...@apache.org>
Authored: Fri Apr 14 12:07:40 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Apr 14 12:07:40 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java   | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0cd4cec5/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index 8f56b10..560ae64 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -432,6 +432,9 @@ public class ThriftServer extends Configured implements Tool {
       throw new RuntimeException("Could not parse the value provided for the port option", e);
     }
 
+    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
+    int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
+
     // Local hostname and user name,
     // used only if QOP is configured.
     String host = null;


[38/50] [abbrv] hbase git commit: HBASE-17090 Redundant exclusion of jruby-complete in pom of hbase-spark

Posted by sy...@apache.org.
HBASE-17090 Redundant exclusion of jruby-complete in pom of hbase-spark

Signed-off-by: Michael 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/e2a74615
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e2a74615
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e2a74615

Branch: refs/heads/hbase-12439
Commit: e2a746152ca8c02c18214f0b5180ed8dcc84e947
Parents: 9dd5cda
Author: Xiang Li <wa...@gmail.com>
Authored: Fri Apr 14 16:15:42 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Apr 14 08:08:42 2017 -0700

----------------------------------------------------------------------
 hbase-spark/pom.xml | 24 ------------------------
 1 file changed, 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e2a74615/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index a7997f1..1afae85 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -290,10 +290,6 @@
                     <artifactId>thrift</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.slf4j</groupId>
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
@@ -338,10 +334,6 @@
                     <artifactId>jasper-compiler</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.jboss.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>
@@ -382,10 +374,6 @@
                     <artifactId>thrift</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.slf4j</groupId>
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
@@ -430,10 +418,6 @@
                     <artifactId>jasper-compiler</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.jboss.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>
@@ -460,10 +444,6 @@
                     <artifactId>thrift</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.slf4j</groupId>
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
@@ -508,10 +488,6 @@
                     <artifactId>jasper-compiler</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>org.jruby</groupId>
-                    <artifactId>jruby-complete</artifactId>
-                </exclusion>
-                <exclusion>
                     <groupId>org.jboss.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>


[15/50] [abbrv] hbase git commit: HBASE-17857 Remove IS annotations from IA.Public classes

Posted by sy...@apache.org.
HBASE-17857 Remove IS annotations from IA.Public classes


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

Branch: refs/heads/hbase-12439
Commit: a66d491892514fd4a188d6ca87d6260d8ae46184
Parents: 910b680
Author: zhangduo <zh...@apache.org>
Authored: Tue Apr 4 20:30:10 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 5 15:34:06 2017 +0800

----------------------------------------------------------------------
 .../hbase/classification/InterfaceAudience.java |   1 -
 .../classification/InterfaceStability.java      |   1 -
 .../hadoop/hbase/CallDroppedException.java      |   4 +-
 .../hadoop/hbase/CallQueueTooBigException.java  |   4 +-
 .../hadoop/hbase/ClockOutOfSyncException.java   |   2 -
 .../org/apache/hadoop/hbase/ClusterStatus.java  |   2 -
 .../hadoop/hbase/DoNotRetryIOException.java     |   2 -
 .../hadoop/hbase/DroppedSnapshotException.java  |   2 -
 .../apache/hadoop/hbase/HColumnDescriptor.java  |   2 -
 .../org/apache/hadoop/hbase/HRegionInfo.java    |   2 -
 .../apache/hadoop/hbase/HRegionLocation.java    |   2 -
 .../apache/hadoop/hbase/HTableDescriptor.java   |   2 -
 .../hbase/InvalidFamilyOperationException.java  |   2 -
 .../apache/hadoop/hbase/KeepDeletedCells.java   |   2 -
 .../hadoop/hbase/MasterNotRunningException.java |   2 -
 .../hadoop/hbase/MemoryCompactionPolicy.java    |   2 -
 .../hadoop/hbase/MultiActionResultTooLarge.java |   2 -
 .../hadoop/hbase/NamespaceExistException.java   |   2 -
 .../hbase/NamespaceNotFoundException.java       |   2 -
 .../hbase/NotAllMetaRegionsOnlineException.java |   2 -
 .../hadoop/hbase/NotServingRegionException.java |   2 -
 .../hadoop/hbase/PleaseHoldException.java       |   2 -
 .../apache/hadoop/hbase/RegionException.java    |   2 -
 .../org/apache/hadoop/hbase/RegionLoad.java     |   2 -
 .../hadoop/hbase/RegionTooBusyException.java    |   2 -
 .../hbase/ReplicationPeerNotFoundException.java |   4 +-
 .../hadoop/hbase/RetryImmediatelyException.java |   2 -
 .../org/apache/hadoop/hbase/ServerLoad.java     |   2 -
 .../hadoop/hbase/TableExistsException.java      |   2 -
 .../hadoop/hbase/TableInfoMissingException.java |   2 -
 .../hadoop/hbase/TableNotDisabledException.java |   2 -
 .../hadoop/hbase/TableNotEnabledException.java  |   2 -
 .../hadoop/hbase/TableNotFoundException.java    |   2 -
 .../hadoop/hbase/UnknownRegionException.java    |   2 -
 .../hadoop/hbase/UnknownScannerException.java   |   2 -
 .../hbase/ZooKeeperConnectionException.java     |   2 -
 .../org/apache/hadoop/hbase/client/Admin.java   |   2 -
 .../org/apache/hadoop/hbase/client/Append.java  |   2 -
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |   2 -
 .../hadoop/hbase/client/AsyncConnection.java    |   4 +-
 .../apache/hadoop/hbase/client/AsyncTable.java  |   2 -
 .../hadoop/hbase/client/AsyncTableBase.java     |   2 -
 .../hadoop/hbase/client/AsyncTableBuilder.java  |   2 -
 .../hbase/client/AsyncTableRegionLocator.java   |   2 -
 .../apache/hadoop/hbase/client/Attributes.java  |   2 -
 .../hadoop/hbase/client/BufferedMutator.java    |   3 -
 .../hbase/client/BufferedMutatorParams.java     |   2 -
 .../apache/hadoop/hbase/client/CompactType.java |   4 +-
 .../hadoop/hbase/client/CompactionState.java    |   2 -
 .../apache/hadoop/hbase/client/Connection.java  |   2 -
 .../hadoop/hbase/client/ConnectionFactory.java  |   2 -
 .../apache/hadoop/hbase/client/Consistency.java |   2 -
 .../org/apache/hadoop/hbase/client/Delete.java  |   2 -
 .../hbase/client/DoNotRetryRegionException.java |   2 -
 .../apache/hadoop/hbase/client/Durability.java  |   2 -
 .../org/apache/hadoop/hbase/client/Get.java     |   2 -
 .../hadoop/hbase/client/HTableMultiplexer.java  |   3 -
 .../apache/hadoop/hbase/client/Increment.java   |   2 -
 .../hadoop/hbase/client/IsolationLevel.java     |   2 -
 .../hadoop/hbase/client/MasterSwitchType.java   |   4 +-
 .../hbase/client/MobCompactPartitionPolicy.java |   2 -
 .../apache/hadoop/hbase/client/Mutation.java    |   2 -
 .../client/NoServerForRegionException.java      |   2 -
 .../apache/hadoop/hbase/client/Operation.java   |   2 -
 .../hbase/client/OperationWithAttributes.java   |   2 -
 .../org/apache/hadoop/hbase/client/Put.java     |   2 -
 .../org/apache/hadoop/hbase/client/Query.java   |   4 +-
 .../hadoop/hbase/client/RawAsyncTable.java      |   4 -
 .../hbase/client/RawScanResultConsumer.java     |   6 +-
 .../hadoop/hbase/client/RegionLoadStats.java    |   2 -
 .../hadoop/hbase/client/RegionLocator.java      |   2 -
 .../hbase/client/RegionOfflineException.java    |   2 -
 .../hadoop/hbase/client/RequestController.java  |   4 -
 .../hbase/client/RequestControllerFactory.java  |   2 -
 .../org/apache/hadoop/hbase/client/Result.java  |   2 -
 .../hadoop/hbase/client/ResultScanner.java      |   2 -
 .../hbase/client/RetriesExhaustedException.java |   2 -
 .../RetriesExhaustedWithDetailsException.java   |   2 -
 .../org/apache/hadoop/hbase/client/Row.java     |   2 -
 .../apache/hadoop/hbase/client/RowAccess.java   |   2 -
 .../hadoop/hbase/client/RowMutations.java       |   2 -
 .../hadoop/hbase/client/RowTooBigException.java |   2 -
 .../hadoop/hbase/client/RpcRetryingCaller.java  |   4 +-
 .../org/apache/hadoop/hbase/client/Scan.java    |   3 -
 .../hadoop/hbase/client/ScanResultConsumer.java |   2 -
 .../client/ShortCircuitMasterConnection.java    |   2 -
 .../hbase/client/SnapshotDescription.java       |   2 -
 .../hadoop/hbase/client/SnapshotType.java       |   4 +-
 .../hbase/client/SyncCoprocessorRpcChannel.java |   4 +-
 .../org/apache/hadoop/hbase/client/Table.java   |   2 -
 .../hadoop/hbase/client/TableBuilder.java       |   2 -
 .../hbase/client/UnmodifyableHRegionInfo.java   |   2 -
 .../client/UnmodifyableHTableDescriptor.java    |   2 -
 .../hbase/client/WrongRowIOException.java       |   2 -
 .../client/backoff/ClientBackoffPolicy.java     |   4 +-
 .../backoff/ExponentialClientBackoffPolicy.java |   4 +-
 .../hadoop/hbase/client/coprocessor/Batch.java  |   4 -
 .../hbase/client/metrics/ScanMetrics.java       |   2 -
 .../client/metrics/ServerSideScanMetrics.java   |   2 -
 .../client/replication/ReplicationAdmin.java    |   2 -
 .../hbase/client/replication/TableCFs.java      |   4 +-
 .../client/security/SecurityCapability.java     |   2 -
 .../coprocessor/BypassCoprocessorException.java |   4 +-
 .../hbase/coprocessor/CoprocessorException.java |   2 -
 .../exceptions/ConnectionClosingException.java  |   2 -
 .../exceptions/FailedSanityCheckException.java  |   2 -
 .../hbase/exceptions/MergeRegionException.java  |   2 -
 .../exceptions/PreemptiveFastFailException.java |   4 +-
 .../exceptions/RegionInRecoveryException.java   |   2 -
 .../exceptions/RequestTooBigException.java      |   2 -
 .../hbase/exceptions/ScannerResetException.java |   2 -
 .../exceptions/UnknownProtocolException.java    |   2 -
 .../hadoop/hbase/filter/BinaryComparator.java   |   2 -
 .../hbase/filter/BinaryPrefixComparator.java    |   2 -
 .../hadoop/hbase/filter/BitComparator.java      |   3 -
 .../hbase/filter/ColumnCountGetFilter.java      |   2 -
 .../hbase/filter/ColumnPaginationFilter.java    |   2 -
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |   2 -
 .../hadoop/hbase/filter/ColumnRangeFilter.java  |   2 -
 .../hadoop/hbase/filter/CompareFilter.java      |   3 -
 .../hbase/filter/DependentColumnFilter.java     |   2 -
 .../hadoop/hbase/filter/FamilyFilter.java       |   2 -
 .../org/apache/hadoop/hbase/filter/Filter.java  |   3 -
 .../apache/hadoop/hbase/filter/FilterList.java  |   3 -
 .../hadoop/hbase/filter/FirstKeyOnlyFilter.java |   2 -
 .../FirstKeyValueMatchingQualifiersFilter.java  |   2 -
 .../hadoop/hbase/filter/FuzzyRowFilter.java     |   2 -
 .../hbase/filter/InclusiveStopFilter.java       |   2 -
 .../filter/IncompatibleFilterException.java     |   2 -
 .../hbase/filter/InvalidRowFilterException.java |   2 -
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |   2 -
 .../hadoop/hbase/filter/LongComparator.java     |   2 -
 .../hbase/filter/MultiRowRangeFilter.java       |   3 -
 .../filter/MultipleColumnPrefixFilter.java      |   2 -
 .../hadoop/hbase/filter/NullComparator.java     |   2 -
 .../apache/hadoop/hbase/filter/PageFilter.java  |   2 -
 .../hadoop/hbase/filter/ParseConstants.java     |   2 -
 .../apache/hadoop/hbase/filter/ParseFilter.java |   2 -
 .../hadoop/hbase/filter/PrefixFilter.java       |   2 -
 .../hadoop/hbase/filter/QualifierFilter.java    |   2 -
 .../hadoop/hbase/filter/RandomRowFilter.java    |   2 -
 .../hbase/filter/RegexStringComparator.java     |   3 -
 .../apache/hadoop/hbase/filter/RowFilter.java   |   2 -
 .../filter/SingleColumnValueExcludeFilter.java  |   2 -
 .../hbase/filter/SingleColumnValueFilter.java   |   2 -
 .../apache/hadoop/hbase/filter/SkipFilter.java  |   2 -
 .../hbase/filter/SubstringComparator.java       |   2 -
 .../hadoop/hbase/filter/TimestampsFilter.java   |   2 -
 .../apache/hadoop/hbase/filter/ValueFilter.java |   2 -
 .../hadoop/hbase/filter/WhileMatchFilter.java   |   2 -
 .../hadoop/hbase/ipc/BadAuthException.java      |   2 -
 .../hbase/ipc/CallCancelledException.java       |   2 -
 .../hadoop/hbase/ipc/CallTimeoutException.java  |   2 -
 .../hbase/ipc/CallerDisconnectedException.java  |   2 -
 .../ipc/CellScannerButNoCodecException.java     |   4 +-
 .../hadoop/hbase/ipc/CoprocessorRpcChannel.java |   4 +-
 .../hadoop/hbase/ipc/FailedServerException.java |   2 -
 .../hbase/ipc/FallbackDisallowedException.java  |   4 +-
 .../hbase/ipc/FatalConnectionException.java     |   2 -
 .../hbase/ipc/NettyRpcClientConfigHelper.java   |   2 -
 .../hbase/ipc/RemoteWithExtrasException.java    |   2 -
 .../hbase/ipc/ServerNotRunningYetException.java |   2 -
 .../hbase/ipc/ServerTooBusyException.java       |   4 +-
 .../hbase/ipc/StoppedRpcClientException.java    |   2 -
 .../ipc/UnsupportedCellCodecException.java      |   2 -
 .../UnsupportedCompressionCodecException.java   |   2 -
 .../hbase/ipc/UnsupportedCryptoException.java   |   2 -
 .../hadoop/hbase/ipc/WrongVersionException.java |   2 -
 .../hbase/quotas/QuotaExceededException.java    |   2 -
 .../apache/hadoop/hbase/quotas/QuotaFilter.java |   2 -
 .../hadoop/hbase/quotas/QuotaRetriever.java     |   4 +-
 .../apache/hadoop/hbase/quotas/QuotaScope.java  |   2 -
 .../hadoop/hbase/quotas/QuotaSettings.java      |   2 -
 .../hbase/quotas/QuotaSettingsFactory.java      |   2 -
 .../apache/hadoop/hbase/quotas/QuotaType.java   |   2 -
 .../hadoop/hbase/quotas/ThrottleType.java       |   2 -
 .../hbase/quotas/ThrottlingException.java       |   3 -
 .../hadoop/hbase/regionserver/BloomType.java    |   2 -
 .../hbase/regionserver/LeaseException.java      |   2 -
 .../NoSuchColumnFamilyException.java            |   2 -
 .../RegionServerAbortedException.java           |   2 -
 .../RegionServerRunningException.java           |   2 -
 .../RegionServerStoppedException.java           |   2 -
 .../regionserver/WrongRegionException.java      |   2 -
 .../wal/FailedLogCloseException.java            |   2 -
 .../wal/FailedSyncBeforeLogCloseException.java  |   2 -
 .../hbase/replication/ReplicationException.java |   2 -
 .../replication/ReplicationPeerConfig.java      |   2 -
 .../replication/ReplicationPeerDescription.java |   4 +-
 .../hbase/security/AccessDeniedException.java   |   2 -
 .../security/access/AccessControlClient.java    |   4 +-
 .../security/access/AccessControlConstants.java |   2 -
 .../hbase/security/access/Permission.java       |   3 -
 .../security/visibility/Authorizations.java     |   2 -
 .../security/visibility/CellVisibility.java     |   2 -
 .../visibility/InvalidLabelException.java       |   2 -
 .../visibility/LabelAlreadyExistsException.java |   2 -
 .../security/visibility/VisibilityClient.java   |   2 -
 .../VisibilityControllerNotReadyException.java  |   2 -
 .../snapshot/CorruptedSnapshotException.java    |   2 -
 .../hbase/snapshot/ExportSnapshotException.java |   2 -
 .../hbase/snapshot/HBaseSnapshotException.java  |   2 -
 .../snapshot/RestoreSnapshotException.java      |   2 -
 .../snapshot/SnapshotCreationException.java     |   2 -
 .../snapshot/SnapshotDoesNotExistException.java |   2 -
 .../hbase/snapshot/SnapshotExistsException.java |   2 -
 .../snapshot/TablePartiallyOpenException.java   |   2 -
 .../snapshot/UnknownSnapshotException.java      |   2 -
 .../hbase/util/FileSystemVersionException.java  |   2 -
 .../apache/hadoop/hbase/util/JsonMapper.java    |   2 -
 .../hbase/TestInterfaceAudienceAnnotations.java | 144 ++++++++++++++-----
 .../java/org/apache/hadoop/hbase/AuthUtil.java  |   2 -
 .../main/java/org/apache/hadoop/hbase/Cell.java |   2 -
 .../java/org/apache/hadoop/hbase/CellUtil.java  |   2 -
 .../org/apache/hadoop/hbase/ChoreService.java   |   2 -
 .../apache/hadoop/hbase/HBaseConfiguration.java |   2 -
 .../apache/hadoop/hbase/HBaseIOException.java   |   4 +-
 .../hadoop/hbase/HBaseInterfaceAudience.java    |   2 -
 .../org/apache/hadoop/hbase/HConstants.java     |   2 -
 .../hadoop/hbase/NamespaceDescriptor.java       |   3 -
 .../org/apache/hadoop/hbase/ProcedureInfo.java  |   4 +-
 .../org/apache/hadoop/hbase/ProcedureState.java |   2 -
 .../org/apache/hadoop/hbase/ScheduledChore.java |   2 -
 .../org/apache/hadoop/hbase/ServerName.java     |   2 -
 .../java/org/apache/hadoop/hbase/Stoppable.java |   2 -
 .../java/org/apache/hadoop/hbase/TableName.java |   2 -
 .../hbase/filter/ByteArrayComparable.java       |   2 -
 .../hadoop/hbase/io/ByteBufferOutputStream.java |   2 -
 .../hadoop/hbase/io/ImmutableBytesWritable.java |   3 -
 .../org/apache/hadoop/hbase/io/TimeRange.java   |   2 -
 .../hadoop/hbase/io/compress/Compression.java   |   2 -
 .../apache/hadoop/hbase/io/crypto/Cipher.java   |   2 -
 .../hadoop/hbase/io/crypto/CipherProvider.java  |   2 -
 .../apache/hadoop/hbase/io/crypto/Context.java  |   2 -
 .../hbase/io/crypto/CryptoCipherProvider.java   |   2 -
 .../hadoop/hbase/io/crypto/Decryptor.java       |   2 -
 .../hbase/io/crypto/DefaultCipherProvider.java  |   2 -
 .../hadoop/hbase/io/crypto/Encryption.java      |   3 -
 .../hadoop/hbase/io/crypto/Encryptor.java       |   2 -
 .../hadoop/hbase/io/crypto/KeyProvider.java     |   2 -
 .../hbase/io/crypto/KeyStoreKeyProvider.java    |   2 -
 .../hbase/io/encoding/DataBlockEncoding.java    |   2 -
 .../org/apache/hadoop/hbase/net/Address.java    |   2 -
 .../hadoop/hbase/rsgroup/RSGroupInfo.java       |   2 -
 .../org/apache/hadoop/hbase/security/User.java  |   2 -
 .../org/apache/hadoop/hbase/types/DataType.java |   2 -
 .../hadoop/hbase/types/FixedLengthWrapper.java  |   2 -
 .../apache/hadoop/hbase/types/OrderedBlob.java  |   2 -
 .../hadoop/hbase/types/OrderedBlobVar.java      |   2 -
 .../hadoop/hbase/types/OrderedBytesBase.java    |   2 -
 .../hadoop/hbase/types/OrderedFloat32.java      |   2 -
 .../hadoop/hbase/types/OrderedFloat64.java      |   2 -
 .../apache/hadoop/hbase/types/OrderedInt16.java |   2 -
 .../apache/hadoop/hbase/types/OrderedInt32.java |   2 -
 .../apache/hadoop/hbase/types/OrderedInt64.java |   2 -
 .../apache/hadoop/hbase/types/OrderedInt8.java  |   2 -
 .../hadoop/hbase/types/OrderedNumeric.java      |   2 -
 .../hadoop/hbase/types/OrderedString.java       |   2 -
 .../org/apache/hadoop/hbase/types/PBType.java   |   2 -
 .../org/apache/hadoop/hbase/types/RawByte.java  |   2 -
 .../org/apache/hadoop/hbase/types/RawBytes.java |   2 -
 .../hadoop/hbase/types/RawBytesFixedLength.java |   2 -
 .../hadoop/hbase/types/RawBytesTerminated.java  |   2 -
 .../apache/hadoop/hbase/types/RawDouble.java    |   2 -
 .../org/apache/hadoop/hbase/types/RawFloat.java |   2 -
 .../apache/hadoop/hbase/types/RawInteger.java   |   2 -
 .../org/apache/hadoop/hbase/types/RawLong.java  |   2 -
 .../org/apache/hadoop/hbase/types/RawShort.java |   2 -
 .../apache/hadoop/hbase/types/RawString.java    |   2 -
 .../hbase/types/RawStringFixedLength.java       |   2 -
 .../hadoop/hbase/types/RawStringTerminated.java |   2 -
 .../org/apache/hadoop/hbase/types/Struct.java   |   2 -
 .../hadoop/hbase/types/StructBuilder.java       |   2 -
 .../hadoop/hbase/types/StructIterator.java      |   2 -
 .../hadoop/hbase/types/TerminatedWrapper.java   |   2 -
 .../org/apache/hadoop/hbase/types/Union2.java   |   2 -
 .../org/apache/hadoop/hbase/types/Union3.java   |   2 -
 .../org/apache/hadoop/hbase/types/Union4.java   |   2 -
 .../org/apache/hadoop/hbase/util/Base64.java    |   5 -
 .../hadoop/hbase/util/ByteBufferUtils.java      |   2 -
 .../org/apache/hadoop/hbase/util/ByteRange.java |   2 -
 .../hadoop/hbase/util/ByteRangeUtils.java       |   2 -
 .../org/apache/hadoop/hbase/util/Bytes.java     |   4 -
 .../org/apache/hadoop/hbase/util/Counter.java   |   2 -
 .../org/apache/hadoop/hbase/util/MD5Hash.java   |   2 -
 .../org/apache/hadoop/hbase/util/Order.java     |   2 -
 .../apache/hadoop/hbase/util/OrderedBytes.java  |   2 -
 .../java/org/apache/hadoop/hbase/util/Pair.java |   2 -
 .../hadoop/hbase/util/PairOfSameType.java       |   2 -
 .../hadoop/hbase/util/PositionedByteRange.java  |   2 -
 .../hbase/util/ReadOnlyByteRangeException.java  |   2 -
 .../hadoop/hbase/util/SimpleByteRange.java      |   2 -
 .../hbase/util/SimpleMutableByteRange.java      |   2 -
 .../hbase/util/SimplePositionedByteRange.java   |   2 -
 .../util/SimplePositionedMutableByteRange.java  |   2 -
 .../apache/hadoop/hbase/util/VersionInfo.java   |   2 -
 .../hadoop/hbase/HBaseCommonTestingUtility.java |   2 -
 .../client/coprocessor/AggregationClient.java   |   2 +-
 .../coprocessor/AsyncAggregationClient.java     |   2 -
 .../org/apache/hadoop/hbase/rest/Constants.java |   2 -
 .../apache/hadoop/hbase/rest/client/Client.java |   2 -
 .../hadoop/hbase/rest/client/Cluster.java       |   2 -
 .../hadoop/hbase/rest/client/RemoteAdmin.java   |   2 -
 .../hadoop/hbase/rest/client/RemoteHTable.java  |   2 -
 .../hadoop/hbase/rest/client/Response.java      |   2 -
 .../rest/filter/RestCsrfPreventionFilter.java   |   1 -
 .../apache/hadoop/hbase/LocalHBaseCluster.java  |   2 -
 .../hbase/client/TableSnapshotScanner.java      |   2 -
 .../hadoop/hbase/client/locking/EntityLock.java |   2 -
 .../hbase/errorhandling/ForeignException.java   |   2 -
 .../hbase/errorhandling/TimeoutException.java   |   2 -
 .../hadoop/hbase/mapred/GroupingTableMap.java   |   2 -
 .../hadoop/hbase/mapred/HRegionPartitioner.java |   4 +-
 .../hadoop/hbase/mapred/IdentityTableMap.java   |   2 -
 .../hbase/mapred/IdentityTableReduce.java       |   2 -
 .../mapred/MultiTableSnapshotInputFormat.java   |   2 -
 .../apache/hadoop/hbase/mapred/RowCounter.java  |   2 -
 .../hadoop/hbase/mapred/TableInputFormat.java   |   2 -
 .../hbase/mapred/TableInputFormatBase.java      |   2 -
 .../apache/hadoop/hbase/mapred/TableMap.java    |   2 -
 .../hadoop/hbase/mapred/TableMapReduceUtil.java |   2 -
 .../hadoop/hbase/mapred/TableOutputFormat.java  |   2 -
 .../hadoop/hbase/mapred/TableRecordReader.java  |   2 -
 .../hbase/mapred/TableRecordReaderImpl.java     |   2 -
 .../apache/hadoop/hbase/mapred/TableReduce.java |   2 -
 .../hbase/mapred/TableSnapshotInputFormat.java  |   2 -
 .../apache/hadoop/hbase/mapred/TableSplit.java  |   2 -
 .../hadoop/hbase/mapreduce/CellCounter.java     |   2 -
 .../hadoop/hbase/mapreduce/CellCreator.java     |   2 -
 .../hadoop/hbase/mapreduce/CopyTable.java       |   2 -
 .../apache/hadoop/hbase/mapreduce/Export.java   |   2 -
 .../hbase/mapreduce/GroupingTableMapper.java    |   2 -
 .../hbase/mapreduce/HFileOutputFormat2.java     |   2 -
 .../hbase/mapreduce/HRegionPartitioner.java     |   2 -
 .../hbase/mapreduce/IdentityTableMapper.java    |   2 -
 .../hbase/mapreduce/IdentityTableReducer.java   |   2 -
 .../apache/hadoop/hbase/mapreduce/Import.java   |   2 -
 .../hadoop/hbase/mapreduce/ImportTsv.java       |   2 -
 .../hbase/mapreduce/KeyValueSerialization.java  |   2 -
 .../hbase/mapreduce/KeyValueSortReducer.java    |   2 -
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |   2 -
 .../hbase/mapreduce/MultiHFileOutputFormat.java |   2 -
 .../hbase/mapreduce/MultiTableInputFormat.java  |   2 -
 .../mapreduce/MultiTableInputFormatBase.java    |   2 -
 .../hbase/mapreduce/MultiTableOutputFormat.java |   2 -
 .../MultiTableSnapshotInputFormat.java          |   2 -
 .../hbase/mapreduce/MutationSerialization.java  |   2 -
 .../hadoop/hbase/mapreduce/PutCombiner.java     |   2 -
 .../hadoop/hbase/mapreduce/PutSortReducer.java  |   2 -
 .../hbase/mapreduce/ResultSerialization.java    |   2 -
 .../hadoop/hbase/mapreduce/RowCounter.java      |   2 -
 .../mapreduce/SimpleTotalOrderPartitioner.java  |   2 -
 .../hbase/mapreduce/TableInputFormat.java       |   2 -
 .../hbase/mapreduce/TableInputFormatBase.java   |   2 -
 .../hbase/mapreduce/TableMapReduceUtil.java     |   2 -
 .../hadoop/hbase/mapreduce/TableMapper.java     |   4 +-
 .../hbase/mapreduce/TableOutputCommitter.java   |   2 -
 .../hbase/mapreduce/TableOutputFormat.java      |   2 -
 .../hbase/mapreduce/TableRecordReader.java      |   2 -
 .../hbase/mapreduce/TableRecordReaderImpl.java  |   2 -
 .../hadoop/hbase/mapreduce/TableReducer.java    |   2 -
 .../mapreduce/TableSnapshotInputFormat.java     |   2 -
 .../hadoop/hbase/mapreduce/TableSplit.java      |   2 -
 .../hadoop/hbase/mapreduce/TextSortReducer.java |   2 -
 .../hbase/mapreduce/TsvImporterMapper.java      |   2 -
 .../hbase/mapreduce/TsvImporterTextMapper.java  |   2 -
 .../mapreduce/VisibilityExpressionResolver.java |   2 -
 .../hadoop/hbase/mapreduce/WALPlayer.java       |   2 -
 .../apache/hadoop/hbase/mob/MobConstants.java   |   2 -
 .../hbase/regionserver/RowTooBigException.java  |   2 +-
 .../hadoop/hbase/security/token/TokenUtil.java  |   2 -
 .../security/visibility/ScanLabelGenerator.java |   2 -
 .../visibility/VisibilityExpEvaluator.java      |   2 -
 .../visibility/VisibilityLabelService.java      |   2 -
 .../hadoop/hbase/snapshot/ExportSnapshot.java   |   2 -
 .../hadoop/hbase/snapshot/SnapshotInfo.java     |   2 -
 .../hadoop/hbase/util/ConfigurationUtil.java    |   2 -
 .../hadoop/hbase/util/EncryptionTest.java       |   2 -
 .../hbase/util/LeaseNotRecoveredException.java  |   2 -
 .../hbase/zookeeper/MiniZooKeeperCluster.java   |   2 -
 .../hadoop/hbase/HBaseTestingUtility.java       |   2 -
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |   2 -
 .../hadoop/hbase/codec/CodecPerformance.java    |   2 -
 .../hbase/spark/BulkLoadPartitioner.scala       |   2 -
 .../hbase/spark/ByteArrayComparable.scala       |   2 -
 .../hadoop/hbase/spark/ByteArrayWrapper.scala   |   2 -
 .../ColumnFamilyQualifierMapKeyWrapper.scala    |   2 -
 .../hbase/spark/FamiliesQualifiersValues.scala  |   2 -
 .../hbase/spark/FamilyHFileWriteOptions.scala   |   2 -
 .../hadoop/hbase/spark/HBaseContext.scala       |   2 -
 .../hbase/spark/HBaseDStreamFunctions.scala     |   2 -
 .../hadoop/hbase/spark/HBaseRDDFunctions.scala  |   2 -
 .../hadoop/hbase/spark/JavaHBaseContext.scala   |   2 -
 .../hadoop/hbase/spark/KeyFamilyQualifier.scala |   2 -
 .../apache/hadoop/hbase/spark/NewHBaseRDD.scala |   2 -
 .../spark/datasources/HBaseSparkConf.scala      |   2 -
 .../datasources/hbase/HBaseTableCatalog.scala   |   2 -
 397 files changed, 137 insertions(+), 877 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java
index 506ef56..8a34a64 100644
--- a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceAudience.java
@@ -43,7 +43,6 @@ import java.lang.annotation.RetentionPolicy;
  * </li></ul>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class InterfaceAudience {
   /**
    * Intended for use by any project or application.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java
index ac20f3a..fbe71d1 100644
--- a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/InterfaceStability.java
@@ -39,7 +39,6 @@ import java.lang.annotation.RetentionPolicy;
  * </ul>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class InterfaceStability {
   /**
    * Can evolve while retaining compatibility for minor release boundaries.;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java
index ed14153..c2d3a7b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Returned to the clients when their request was discarded due to server being overloaded.
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CallDroppedException extends IOException {
   public CallDroppedException() {
     super();
@@ -40,4 +38,4 @@ public class CallDroppedException extends IOException {
   public CallDroppedException(String message) {
     super(message);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java
index 9f8b386..d615d0e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java
@@ -21,11 +21,9 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CallQueueTooBigException extends IOException {
   public CallQueueTooBigException() {
     super();
@@ -36,4 +34,4 @@ public class CallQueueTooBigException extends IOException {
   public CallQueueTooBigException(String message) {
     super(message);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
index b27ae82..9e3f556 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This exception is thrown by the master when a region server clock skew is
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ClockOutOfSyncException extends IOException {
   public ClockOutOfSyncException(String message) {
     super(message);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index aed3af4..c51a437 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -26,7 +26,6 @@ import java.util.Set;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.io.VersionedWritable;
 
@@ -50,7 +49,6 @@ import org.apache.hadoop.io.VersionedWritable;
  * </ul>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ClusterStatus extends VersionedWritable {
   /**
    * Version for object serialization.  Incremented for changes in serialized

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
index 8be2518..225ec4e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Subclass if exception is not meant to be retried: e.g.
  * {@link org.apache.hadoop.hbase.UnknownScannerException}
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class DoNotRetryIOException extends HBaseIOException {
   // TODO: This would be more useful as a marker interface than as a class.
   private static final long serialVersionUID = 1197446454511704139L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
index 1000d7d..2dbc93e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 
 /**
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * persisted into store files.  Response should include replay of wal content.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class DroppedSnapshotException extends IOException {
 
   private static final long serialVersionUID = -5463156580831677374L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index 545ea61..46e97c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -27,7 +27,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.HBaseException;
@@ -49,7 +48,6 @@ import com.google.common.base.Preconditions;
  * It is used as input when creating a table or adding a column.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
   // For future backward compatibility
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index b98d210..bc93cc6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -28,7 +28,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -76,7 +75,6 @@ import org.apache.hadoop.util.StringUtils;
  * previous behavior of a range corresponding to 1 region.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HRegionInfo implements Comparable<HRegionInfo> {
 
   private static final Log LOG = LogFactory.getLog(HRegionInfo.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
index edb53dc..6cf22dd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Addressing;
 
 /**
@@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.util.Addressing;
  * <br>This interface has been marked InterfaceAudience.Public in 0.96 and 0.98.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HRegionLocation implements Comparable<HRegionLocation> {
   private final HRegionInfo regionInfo;
   private final ServerName serverName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 25fd896..ed0659c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -36,7 +36,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.client.Durability;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * when the region split should occur, coprocessors associated with it etc...
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HTableDescriptor implements Comparable<HTableDescriptor> {
 
   private static final Log LOG = LogFactory.getLog(HTableDescriptor.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
index 5d9c2ed..8fb05d5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if a request is table schema modification is requested but
  * made for an invalid family name.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class InvalidFamilyOperationException extends DoNotRetryIOException {
   private static final long serialVersionUID = 1L << 22 - 1L;
   /** default constructor */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java
index d2d92b3..28ed13e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Ways to keep cells marked for delete around.
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * this way for backwards compatibility.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum KeepDeletedCells {
   /** Deleted Cells are not retained. */
   FALSE,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
index ddd03e8..70afee2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
@@ -21,13 +21,11 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if the master is not running
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class MasterNotRunningException extends IOException {
   private static final long serialVersionUID = 1L << 23 - 1L;
   /** default constructor */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java
index 0153f7d..8dc92ae 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Enum describing all possible memory compaction policies
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum MemoryCompactionPolicy {
   /**
    * No memory compaction, when size threshold is exceeded data is flushed to disk

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
index fdff554..90cd2c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Exception thrown when the result needs to be chunked on the server side.
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * retries because some of the multi was a success.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MultiActionResultTooLarge extends RetryImmediatelyException {
   public MultiActionResultTooLarge(String s) {
     super(s);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
index a7ebf0d..10a1ed6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceExistException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a namespace exists but should not
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class NamespaceExistException extends DoNotRetryIOException {
 
   private static final long serialVersionUID = -1582357514338825412L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
index 092290d..f61cdea 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NamespaceNotFoundException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a namespace can not be located
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class NamespaceNotFoundException extends DoNotRetryIOException {
 
   private static final long serialVersionUID = -6673607347330260324L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
index 3cb0f5b..ce6acf7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
@@ -20,13 +20,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when an operation requires the root and all meta regions to be online
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class NotAllMetaRegionsOnlineException extends DoNotRetryIOException {
   private static final long serialVersionUID = 6439786157874827523L;
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
index 8975c74..bca9cd2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * serving.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class NotServingRegionException extends IOException {
   private static final long serialVersionUID = 1L << 17 - 1L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
index a5ae44b..dc6fd4b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/PleaseHoldException.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This exception is thrown by the master when a region server was shut down and
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class PleaseHoldException extends HBaseIOException {
   public PleaseHoldException(String message) {
     super(message);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
index 24ea16c..e0c0346 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionException.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when something happens related to region handling.
  * Subclasses have to be more specific.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RegionException extends HBaseIOException {
   private static final long serialVersionUID = 1473510258071111371L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
index befb2de..d6c028d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
@@ -23,7 +23,6 @@ package org.apache.hadoop.hbase;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.util.Strings;
   * Encapsulates per-region load metrics.
   */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionLoad {
 
   protected ClusterStatusProtos.RegionLoad regionLoadPB;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
index fd5fc26..5d4cad4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown by a region server if it will block and wait to serve a request.
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * region is compacting.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionTooBusyException extends IOException {
   private static final long serialVersionUID = 1728345723728342L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ReplicationPeerNotFoundException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ReplicationPeerNotFoundException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ReplicationPeerNotFoundException.java
index daf7dd5..8ad93f2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ReplicationPeerNotFoundException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ReplicationPeerNotFoundException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a replication peer can not be found
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ReplicationPeerNotFoundException extends DoNotRetryIOException {
 
   private static final long serialVersionUID = 1L;
@@ -33,4 +31,4 @@ public class ReplicationPeerNotFoundException extends DoNotRetryIOException {
   public ReplicationPeerNotFoundException(String peerId) {
     super(peerId);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
index e0b90fd..b97c168 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
@@ -20,10 +20,8 @@ package org.apache.hadoop.hbase;
 
 import java.io.IOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RetryImmediatelyException extends IOException {
   public RetryImmediatelyException(String s) {
     super(s);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index e884e51..8547dfb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -27,7 +27,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor;
@@ -40,7 +39,6 @@ import org.apache.hadoop.hbase.util.Strings;
  * This class is used for exporting current state of load on a RegionServer.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ServerLoad {
   private int stores = 0;
   private int storefiles = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
index 623e8cf..6c0f3bb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a table exists but should not
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableExistsException extends DoNotRetryIOException {
   private static final long serialVersionUID = 1L << 7 - 1L;
   /** default constructor */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
index fa1f970..391fd43 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableInfoMissingException.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * 
@@ -26,7 +25,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  *
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @SuppressWarnings("serial")
 public class TableInfoMissingException extends HBaseIOException {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
index 9b5f728..dcfa857 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Thrown if a table should be offline but is not
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableNotDisabledException extends DoNotRetryIOException {
   private static final long serialVersionUID = 1L << 19 - 1L;
   /** default constructor */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
index 0f78ee6..5de406e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * Thrown if a table should be enabled but is not
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableNotEnabledException extends DoNotRetryIOException {
   private static final long serialVersionUID = 262144L;
   /** default constructor */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
index 8ac5e20..6344cd3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
@@ -19,12 +19,10 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /** Thrown when a table can not be located */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TableNotFoundException extends DoNotRetryIOException {
   private static final long serialVersionUID = 993179627856392526L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
index 2ebba32..e9684ae 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
 
 /**
  * Thrown when we are asked to operate on a region we know nothing about.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class UnknownRegionException extends DoNotRetryRegionException {
   private static final long serialVersionUID = 1968858760475205392L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
index 3e7b22d..8f7d441 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/UnknownScannerException.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 
 /**
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * down and has cancelled all leases.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class UnknownScannerException extends DoNotRetryIOException {
   private static final long serialVersionUID = 993179627856392526L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java
index 422a659..c492a27 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java
@@ -21,13 +21,11 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if the client can't connect to zookeeper
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ZooKeeperConnectionException extends IOException {
   private static final long serialVersionUID = 1L << 23 - 1L;
   /** default constructor */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/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 cc14acd..f2fc9a5 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
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -71,7 +70,6 @@ import org.apache.hadoop.hbase.util.Pair;
  * @since 0.99.0
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface Admin extends Abortable, Closeable {
   int getOperationTimeout();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
index 15497ce..a655c7d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * {@link #add(byte[], byte[], byte[])} method.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Append extends Mutation {
   /**
    * @param returnResults

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index ef7a4f2..ab791c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -27,14 +27,12 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 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.util.Pair;
 
 /**
  *  The asynchronous administrative API for HBase.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface AsyncAdmin {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
index dbe32ca..65005fa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
@@ -23,13 +23,11 @@ import java.util.concurrent.ExecutorService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * The asynchronous version of Connection.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface AsyncConnection extends Closeable {
 
   /**
@@ -105,4 +103,4 @@ public interface AsyncConnection extends Closeable {
    * @return an AsyncAdmin instance for cluster administration
    */
   AsyncAdmin getAdmin();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
index 402ad64..a2e193c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * The asynchronous table for normal users.
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * to provide a {@code ExecutorService}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface AsyncTable extends AsyncTableBase {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
index b5a251b..73ebebb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBase.java
@@ -30,7 +30,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 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.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * from the returned {@link CompletableFuture}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface AsyncTableBase {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java
index 2330855..1cc5f10 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java
@@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * For creating {@link AsyncTable} or {@link RawAsyncTable}.
@@ -32,7 +31,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * AsyncTable/RawAsyncTable instance.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface AsyncTableBuilder<T extends AsyncTableBase> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
index 989e8d9..13434a6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocator.java
@@ -22,7 +22,6 @@ import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * The asynchronous version of RegionLocator.
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * from the returned {@link CompletableFuture}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public interface AsyncTableRegionLocator {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java
index 78d3398..fd36e76 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Attributes.java
@@ -22,10 +22,8 @@ package org.apache.hadoop.hbase.client;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public interface Attributes {
   /**
    * Sets an attribute.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
index cea9304..766c28c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -61,7 +60,6 @@ import java.util.List;
  * @since 1.0.0
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface BufferedMutator extends Closeable {
   /**
    * Key to use setting non-default BufferedMutator implementation in Configuration.
@@ -138,7 +136,6 @@ public interface BufferedMutator extends Closeable {
    * Listens for asynchronous exceptions on a {@link BufferedMutator}.
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   interface ExceptionListener {
     public void onException(RetriesExhaustedWithDetailsException exception,
         BufferedMutator mutator) throws RetriesExhaustedWithDetailsException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java
index 9c901e2..060fc77 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java
@@ -23,13 +23,11 @@ import java.util.concurrent.ExecutorService;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Parameters for instantiating a {@link BufferedMutator}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class BufferedMutatorParams implements Cloneable {
 
   static final int UNSET = -1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java
index 17fec2b..9432378 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Currently, there are only two compact types:
@@ -25,11 +24,10 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * {@code MOB} means do mob files compaction.
  * */
 @InterfaceAudience.Public
-@InterfaceStability.Unstable
 public enum CompactType {
 
   NORMAL    (0),
   MOB       (1);
 
   CompactType(int value) {}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactionState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactionState.java
index b4824ef..8f2a83b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactionState.java
@@ -17,13 +17,11 @@
  */
 package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * POJO representing the compaction state
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum CompactionState {
   NONE, MINOR, MAJOR, MAJOR_AND_MINOR;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
index a8cd296..8eedb79 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A cluster connection encapsulating lower level individual connections to actual servers and
@@ -50,7 +49,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * @since 0.99.0
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface Connection extends Abortable, Closeable {
 
   /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index 64f337a..156a3c8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -26,7 +26,6 @@ import java.util.concurrent.ExecutorService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
@@ -54,7 +53,6 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * @since 0.99.0
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ConnectionFactory {
 
   public static final String HBASE_CLIENT_ASYNC_CONNECTION_IMPL = "hbase.client.async.connection.impl";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Consistency.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Consistency.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Consistency.java
index 39323d6..b6e0531 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Consistency.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Consistency.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Consistency defines the expected consistency level for an operation.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum Consistency {
   // developer note: Do not reorder. Client.proto#Consistency depends on this order
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 0eb1d2b..278ea58 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -69,7 +68,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * timestamp.  The constructor timestamp is not referenced.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Delete extends Mutation implements Comparable<Row> {
   /**
    * Create a Delete operation for the specified row.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DoNotRetryRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DoNotRetryRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DoNotRetryRegionException.java
index 3c5dd28..e0b5ead 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DoNotRetryRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DoNotRetryRegionException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Similar to RegionException, but disables retries.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class DoNotRetryRegionException extends DoNotRetryIOException {
 
   private static final long serialVersionUID = 6907047686199321701L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
index 9b35e04..a3fa1d2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Enum describing the durability guarantees for tables and {@link Mutation}s
  * Note that the items must be sorted in order of increasing durability
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum Durability {
   /* Developer note: Do not rename the enum field names. They are serialized in HTableDescriptor */
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
index 3771aff..c3ddc4b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
@@ -34,7 +34,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.filter.Filter;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.security.access.Permission;
@@ -65,7 +64,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * To add a filter, call {@link #setFilter(Filter) setFilter}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class Get extends Query
   implements Row, Comparable<Row> {
   private static final Log LOG = LogFactory.getLog(Get.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index f3a58ad..313125c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
 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;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
@@ -67,7 +66,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  * This class is thread safe.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HTableMultiplexer {
   private static final Log LOG = LogFactory.getLog(HTableMultiplexer.class.getName());
 
@@ -271,7 +269,6 @@ public class HTableMultiplexer {
    * in total or on per region server basis.
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public static class HTableMultiplexerStatus {
     private long totalFailedPutCounter;
     private long totalBufferedPutCounter;


[26/50] [abbrv] hbase git commit: HBASE-17863-addendum: Reverted the order of updateStoreOnExec() and store.isRunning() in execProcedure()

Posted by sy...@apache.org.
HBASE-17863-addendum: Reverted the order of updateStoreOnExec() and store.isRunning() in execProcedure()

Change-Id: I1c9d5ee264f4f593a6b2a09011853ab63693f677


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

Branch: refs/heads/hbase-12439
Commit: 59e8b8e2ba4d403d042fe4cc02f8f9f80aad67af
Parents: 18c5ecf
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Fri Apr 7 14:01:37 2017 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Apr 7 16:13:37 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/procedure2/ProcedureExecutor.java       | 11 ++++++++---
 .../hadoop/hbase/procedure2/ProcedureTestingUtility.java |  2 +-
 .../procedure2/store/wal/TestWALProcedureStore.java      |  3 +--
 3 files changed, 10 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59e8b8e2/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 8832637..43f5839 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -1373,12 +1373,17 @@ public class ProcedureExecutor<TEnvironment> {
         return;
       }
 
-      // if the store is not running we are aborting
-      if (!store.isRunning()) return;
-
+      // TODO: The code here doesn't check if store is running before persisting to the store as
+      // it relies on the method call below to throw RuntimeException to wind up the stack and
+      // executor thread to stop. The statement following the method call below seems to check if
+      // store is not running, to prevent scheduling children procedures, re-execution or yield
+      // of this procedure. This may need more scrutiny and subsequent cleanup in future
       // Commit the transaction
       updateStoreOnExec(procStack, procedure, subprocs);
 
+      // if the store is not running we are aborting
+      if (!store.isRunning()) return;
+
       // if the procedure is kind enough to pass the slot to someone else, yield
       if (procedure.isRunnable() && !suspended &&
           procedure.isYieldAfterExecutionStep(getEnvironment())) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/59e8b8e2/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 1f4244a..dd3c8f4 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -408,7 +408,7 @@ public class ProcedureTestingUtility {
       addStackIndex(index);
     }
 
-    public void setFinishedState() {
+    public void setSuccessState() {
       setState(ProcedureState.SUCCESS);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/59e8b8e2/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
index f8c3486..525a663 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.logging.Log;
@@ -785,7 +784,7 @@ public class TestWALProcedureStore {
 
     // back to A
     a.addStackId(5);
-    a.setFinishedState();
+    a.setSuccessState();
     procStore.delete(a, new long[] { b.getProcId(), c.getProcId() });
     restartAndAssert(3, 0, 1, 0);
   }


[32/50] [abbrv] hbase git commit: HBASE-17905 [hbase-spark] bulkload does not work when table not exist

Posted by sy...@apache.org.
HBASE-17905 [hbase-spark] bulkload does not work when table not exist

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: d7ddc79198679d8c642e7d8ad5141ba518f8d9f3
Parents: 02da5a6
Author: Yi Liang <ea...@gmail.com>
Authored: Tue Apr 11 17:04:40 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Apr 11 17:18:49 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/spark/BulkLoadPartitioner.scala  | 13 ++++++++-----
 .../apache/hadoop/hbase/spark/HBaseContext.scala  | 18 +++++++++++++++++-
 2 files changed, 25 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d7ddc791/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
index ab4fc41..022c933 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala
@@ -33,8 +33,8 @@ import org.apache.spark.Partitioner
 @InterfaceAudience.Public
 class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
   extends Partitioner {
-
-  override def numPartitions: Int = startKeys.length
+  // when table not exist, startKeys = Byte[0][]
+  override def numPartitions: Int = if (startKeys.length == 0) 1 else startKeys.length
 
   override def getPartition(key: Any): Int = {
 
@@ -53,8 +53,11 @@ class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
         case _ =>
           key.asInstanceOf[Array[Byte]]
       }
-    val partition = util.Arrays.binarySearch(startKeys, rowKey, comparator)
-    if (partition < 0) partition * -1 + -2
-    else partition
+    var partition = util.Arrays.binarySearch(startKeys, rowKey, comparator)
+    if (partition < 0)
+      partition = partition * -1 + -2
+    if (partition < 0)
+      partition = 0
+    partition
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7ddc791/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 e2891db..1948bd3 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
@@ -48,7 +48,7 @@ import org.apache.spark.streaming.dstream.DStream
 import java.io._
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod
-import org.apache.hadoop.fs.{Path, FileSystem}
+import org.apache.hadoop.fs.{Path, FileAlreadyExistsException, FileSystem}
 import scala.collection.mutable
 
 /**
@@ -620,9 +620,17 @@ class HBaseContext(@transient sc: SparkContext,
                   compactionExclude: Boolean = false,
                   maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):
   Unit = {
+    val stagingPath = new Path(stagingDir)
+    val fs = stagingPath.getFileSystem(config)
+    if (fs.exists(stagingPath)) {
+      throw new FileAlreadyExistsException("Path " + stagingDir + " already exists")
+    }
     val conn = HBaseConnectionCache.getConnection(config)
     val regionLocator = conn.getRegionLocator(tableName)
     val startKeys = regionLocator.getStartKeys
+    if (startKeys.length == 0) {
+      logInfo("Table " + tableName.toString + " was not found")
+    }
     val defaultCompressionStr = config.get("hfile.compression",
       Compression.Algorithm.NONE.getName)
     val hfileCompression = HFileWriterImpl
@@ -743,9 +751,17 @@ class HBaseContext(@transient sc: SparkContext,
                   compactionExclude: Boolean = false,
                   maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):
   Unit = {
+    val stagingPath = new Path(stagingDir)
+    val fs = stagingPath.getFileSystem(config)
+    if (fs.exists(stagingPath)) {
+      throw new FileAlreadyExistsException("Path " + stagingDir + " already exists")
+    }
     val conn = HBaseConnectionCache.getConnection(config)
     val regionLocator = conn.getRegionLocator(tableName)
     val startKeys = regionLocator.getStartKeys
+    if (startKeys.length == 0) {
+      logInfo("Table " + tableName.toString + " was not found")
+    }
     val defaultCompressionStr = config.get("hfile.compression",
       Compression.Algorithm.NONE.getName)
     val defaultCompression = HFileWriterImpl


[49/50] [abbrv] hbase git commit: Revert "HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)"

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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 095f4bd..d56d6ec 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
@@ -116,7 +116,6 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterAllFilter;
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.NullComparator;
@@ -4932,7 +4931,6 @@ public class TestHRegion {
       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
       throws IOException {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
     return initHRegion(tableName, startKey, stopKey, isReadOnly,

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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 6eed7df..0054642 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
@@ -153,7 +153,7 @@ public class TestHRegionReplayEvents {
     }
 
     time = System.currentTimeMillis();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+
     primaryHri = new HRegionInfo(htd.getTableName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
       false, time, 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index 1768801..37a7664 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -48,30 +48,30 @@ import static org.junit.Assert.assertTrue;
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestMemStoreChunkPool {
   private final static Configuration conf = new Configuration();
-  private static ChunkCreator chunkCreator;
+  private static MemStoreChunkPool chunkPool;
   private static boolean chunkPoolDisabledBeforeTest;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    chunkPoolDisabledBeforeTest = ChunkCreator.chunkPoolDisabled;
-    ChunkCreator.chunkPoolDisabled = false;
+    chunkPoolDisabledBeforeTest = MemStoreChunkPool.chunkPoolDisabled;
+    MemStoreChunkPool.chunkPoolDisabled = false;
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
-      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-    assertTrue(chunkCreator != null);
+    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
+        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
+    assertTrue(chunkPool != null);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    ChunkCreator.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
+    MemStoreChunkPool.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
   }
 
   @Before
   public void tearDown() throws Exception {
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
   }
 
   @Test
@@ -90,7 +90,7 @@ public class TestMemStoreChunkPool {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        expectedOff = 8;
+        expectedOff = 0;
         lastBuffer = newKv.getBuffer();
       }
       assertEquals(expectedOff, newKv.getOffset());
@@ -100,14 +100,14 @@ public class TestMemStoreChunkPool {
     }
     // chunks will be put back to pool after close
     mslab.close();
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
     // reconstruct mslab
     mslab = new MemStoreLABImpl(conf);
     // chunk should be got from the pool, so we can reuse it.
     KeyValue kv = new KeyValue(rk, cf, q, new byte[10]);
     mslab.copyCellInto(kv);
-    assertEquals(chunkCount - 1, chunkCreator.getPoolSize());
+    assertEquals(chunkCount - 1, chunkPool.getPoolSize());
   }
 
   @Test
@@ -143,7 +143,7 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -189,16 +189,16 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
     snapshot = memstore.snapshot();
@@ -218,20 +218,20 @@ public class TestMemStoreChunkPool {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   @Test
   public void testPutbackChunksMultiThreaded() throws Exception {
+    MemStoreChunkPool oldPool = MemStoreChunkPool.GLOBAL_INSTANCE;
     final int maxCount = 10;
     final int initialCount = 5;
-    final int chunkSize = 40;
+    final int chunkSize = 30;
     final int valSize = 7;
-    ChunkCreator oldCreator = ChunkCreator.getInstance();
-    ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, null);
-    assertEquals(initialCount, newCreator.getPoolSize());
-    assertEquals(maxCount, newCreator.getMaxCount());
-    ChunkCreator.INSTANCE = newCreator;// Replace the global ref with the new one we created.
+    MemStoreChunkPool pool = new MemStoreChunkPool(chunkSize, maxCount, initialCount, 1, false);
+    assertEquals(initialCount, pool.getPoolSize());
+    assertEquals(maxCount, pool.getMaxCount());
+    MemStoreChunkPool.GLOBAL_INSTANCE = pool;// Replace the global ref with the new one we created.
                                              // Used it for the testing. Later in finally we put
                                              // back the original
     final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
@@ -258,9 +258,9 @@ public class TestMemStoreChunkPool {
       t1.join();
       t2.join();
       t3.join();
-      assertTrue(newCreator.getPoolSize() <= maxCount);
+      assertTrue(pool.getPoolSize() <= maxCount);
     } finally {
-      ChunkCreator.INSTANCE = oldCreator;
+      MemStoreChunkPool.GLOBAL_INSTANCE = oldPool;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
index 6696e43..141b802 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
@@ -63,8 +63,8 @@ public class TestMemStoreLAB {
   public static void setUpBeforeClass() throws Exception {
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, globalMemStoreLimit,
-      0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+    MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT,
+        MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
   }
 
   /**
@@ -76,7 +76,6 @@ public class TestMemStoreLAB {
     MemStoreLAB mslab = new MemStoreLABImpl();
     int expectedOff = 0;
     ByteBuffer lastBuffer = null;
-    long lastChunkId = -1;
     // 100K iterations by 0-1K alloc -> 50MB expected
     // should be reasonable for unit test and also cover wraparound
     // behavior
@@ -86,13 +85,8 @@ public class TestMemStoreLAB {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        expectedOff = Bytes.SIZEOF_LONG;
+        expectedOff = 0;
         lastBuffer = newKv.getBuffer();
-        long chunkId = newKv.getBuffer().getLong(0);
-        assertTrue("chunkid should be different", chunkId != lastChunkId);
-        lastChunkId = chunkId;
       }
       assertEquals(expectedOff, newKv.getOffset());
       assertTrue("Allocation overruns buffer",
@@ -142,21 +136,23 @@ public class TestMemStoreLAB {
       };
       ctx.addThread(t);
     }
-
+    
     ctx.startThreads();
     while (totalAllocated.get() < 50*1024*1024 && ctx.shouldRun()) {
       Thread.sleep(10);
     }
     ctx.stop();
+    
     // Partition the allocations by the actual byte[] they point into,
     // make sure offsets are unique for each chunk
     Map<ByteBuffer, Map<Integer, AllocRecord>> mapsByChunk =
       Maps.newHashMap();
-
+    
     int sizeCounted = 0;
     for (AllocRecord rec : Iterables.concat(allocations)) {
       sizeCounted += rec.size;
       if (rec.size == 0) continue;
+      
       Map<Integer, AllocRecord> mapForThisByteArray =
         mapsByChunk.get(rec.alloc);
       if (mapForThisByteArray == null) {
@@ -171,9 +167,7 @@ public class TestMemStoreLAB {
     
     // Now check each byte array to make sure allocations don't overlap
     for (Map<Integer, AllocRecord> allocsInChunk : mapsByChunk.values()) {
-      // since we add the chunkID at the 0th offset of the chunk and the
-      // chunkid is a long we need to account for those 8 bytes
-      int expectedOff = Bytes.SIZEOF_LONG;
+      int expectedOff = 0;
       for (AllocRecord alloc : allocsInChunk.values()) {
         assertEquals(expectedOff, alloc.offset);
         assertTrue("Allocation overruns buffer",
@@ -181,6 +175,7 @@ public class TestMemStoreLAB {
         expectedOff += alloc.size;
       }
     }
+
   }
 
   /**
@@ -199,7 +194,7 @@ public class TestMemStoreLAB {
     // set chunk size to default max alloc size, so we could easily trigger chunk retirement
     conf.setLong(MemStoreLABImpl.CHUNK_SIZE_KEY, MemStoreLABImpl.MAX_ALLOC_DEFAULT);
     // reconstruct mslab
-    ChunkCreator.clearDisableFlag();
+    MemStoreChunkPool.clearDisableFlag();
     mslab = new MemStoreLABImpl(conf);
     // launch multiple threads to trigger frequent chunk retirement
     List<Thread> threads = new ArrayList<>();
@@ -228,8 +223,6 @@ public class TestMemStoreLAB {
     }
     // close the mslab
     mslab.close();
-    // none of the chunkIds would have been returned back
-    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() != 0);
     // make sure all chunks reclaimed or removed from chunk queue
     int queueLength = mslab.getPooledChunks().size();
     assertTrue("All chunks in chunk queue should be reclaimed or removed"

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
deleted file mode 100644
index f38a75e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.lang.management.ManagementFactory;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestMemstoreLABWithoutPool {
-  private final static Configuration conf = new Configuration();
-
-  private static final byte[] rk = Bytes.toBytes("r1");
-  private static final byte[] cf = Bytes.toBytes("f");
-  private static final byte[] q = Bytes.toBytes("q");
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
-        .getMax() * 0.8);
-    // disable pool
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT + Bytes.SIZEOF_LONG, false, globalMemStoreLimit,
-      0.0f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-  }
-
-  /**
-   * Test a bunch of random allocations
-   */
-  @Test
-  public void testLABRandomAllocation() {
-    Random rand = new Random();
-    MemStoreLAB mslab = new MemStoreLABImpl();
-    int expectedOff = 0;
-    ByteBuffer lastBuffer = null;
-    long lastChunkId = -1;
-    // 100K iterations by 0-1K alloc -> 50MB expected
-    // should be reasonable for unit test and also cover wraparound
-    // behavior
-    for (int i = 0; i < 100000; i++) {
-      int valSize = rand.nextInt(1000);
-      KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
-      int size = KeyValueUtil.length(kv);
-      ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
-      if (newKv.getBuffer() != lastBuffer) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        expectedOff = Bytes.SIZEOF_LONG;
-        lastBuffer = newKv.getBuffer();
-        long chunkId = newKv.getBuffer().getLong(0);
-        assertTrue("chunkid should be different", chunkId != lastChunkId);
-        lastChunkId = chunkId;
-      }
-      assertEquals(expectedOff, newKv.getOffset());
-      assertTrue("Allocation overruns buffer",
-          newKv.getOffset() + size <= newKv.getBuffer().capacity());
-      expectedOff += size;
-    }
-  }
-
-  /**
-   * Test frequent chunk retirement with chunk pool triggered by lots of threads, making sure
-   * there's no memory leak (HBASE-16195)
-   * @throws Exception if any error occurred
-   */
-  @Test
-  public void testLABChunkQueueWithMultipleMSLABs() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    MemStoreLABImpl[] mslab = new MemStoreLABImpl[10];
-    for (int i = 0; i < 10; i++) {
-      mslab[i] = new MemStoreLABImpl(conf);
-    }
-    // launch multiple threads to trigger frequent chunk retirement
-    List<Thread> threads = new ArrayList<>();
-    // create smaller sized kvs
-    final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
-        new byte[0]);
-    for (int i = 0; i < 10; i++) {
-      for (int j = 0; j < 10; j++) {
-        threads.add(getChunkQueueTestThread(mslab[i], "testLABChunkQueue-" + j, kv));
-      }
-    }
-    for (Thread thread : threads) {
-      thread.start();
-    }
-    // let it run for some time
-    Thread.sleep(3000);
-    for (Thread thread : threads) {
-      thread.interrupt();
-    }
-    boolean threadsRunning = true;
-    boolean alive = false;
-    while (threadsRunning) {
-      alive = false;
-      for (Thread thread : threads) {
-        if (thread.isAlive()) {
-          alive = true;
-          break;
-        }
-      }
-      if (!alive) {
-        threadsRunning = false;
-      }
-    }
-    // close the mslab
-    for (int i = 0; i < 10; i++) {
-      mslab[i].close();
-    }
-    // all of the chunkIds would have been returned back
-    assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() == 0);
-  }
-
-  private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,
-      Cell cellToCopyInto) {
-    Thread thread = new Thread() {
-      boolean stopped = false;
-
-      @Override
-      public void run() {
-        while (!stopped) {
-          // keep triggering chunk retirement
-          mslab.copyCellInto(cellToCopyInto);
-        }
-      }
-
-      @Override
-      public void interrupt() {
-        this.stopped = true;
-      }
-    };
-    thread.setName(threadName);
-    thread.setDaemon(true);
-    return thread;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 7160e5e..4315bd4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -108,7 +108,6 @@ public class TestRecoveredEdits {
       }
     };
     Path hbaseRootDir = TEST_UTIL.getDataTestDir();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
     Path tableDir = FSUtils.getTableDir(hbaseRootDir, htd.getTableName());
     HRegionFileSystem hrfs =

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index ad56081..5d11c0e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -83,7 +83,6 @@ public class TestRegionIncrement {
   private HRegion getRegion(final Configuration conf, final String tableName) throws IOException {
     WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
       TEST_UTIL.getDataTestDir().toString(), conf);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, conf,
       false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 5355c77..0d339b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -178,7 +178,6 @@ public class TestStore {
     } else {
       htd.addFamily(hcd);
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 99dd00d..3cdb227 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -111,7 +111,6 @@ public class TestStoreFileRefresherChore {
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region =
         new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
             conf, htd, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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 51260a6..4f247b0 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
@@ -588,7 +588,6 @@ public class TestWALLockup {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index e63bad9..994779f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -98,7 +98,6 @@ public class TestWALMonotonicallyIncreasingSeqId {
     FSUtils.setRootDir(walConf, tableDir);
     this.walConf = walConf;
     wals = new WALFactory(walConf, null, "log_" + replicaId);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region = new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(),
         info.getTable().getNamespace()), conf, htd, null);
     region.initialize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index 057b9bf..f976b49 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -37,9 +37,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -290,7 +288,6 @@ public class TestDurability {
           throw new IOException("Failed delete of " + path);
         }
       }
-      ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
       return HRegion.createHRegion(info, path, CONF, htd, log);
     }
 


[35/50] [abbrv] hbase git commit: HBASE-17897 StripeStoreFileManager#nonOpenRowCompare use the wrong comparison function

Posted by sy...@apache.org.
HBASE-17897 StripeStoreFileManager#nonOpenRowCompare use the wrong comparison function


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

Branch: refs/heads/hbase-12439
Commit: cf3215d343d5c412735f51c68dc3686087736a86
Parents: 23249eb
Author: CHIA-PING TSAI <ch...@gmail.com>
Authored: Mon Apr 10 17:03:47 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Wed Apr 12 11:54:42 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/StripeStoreFileManager.java      | 2 +-
 .../hadoop/hbase/regionserver/TestStripeStoreFileManager.java  | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cf3215d3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
index 4a719f3..ef86e2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
@@ -562,7 +562,7 @@ public class StripeStoreFileManager
    */
   private final int nonOpenRowCompare(byte[] k1, byte[] k2) {
     assert !isOpen(k1) && !isOpen(k2);
-    return cellComparator.compareRows(new KeyOnlyKeyValue(k1), k2, 0, k2.length);
+    return Bytes.compareTo(k1, k2);
   }
 
   private final int nonOpenRowCompare(Cell k1, byte[] k2) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf3215d3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
index a6ce270..ca13263 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
@@ -58,9 +58,9 @@ public class TestStripeStoreFileManager {
   private static final Path CFDIR = HStore.getStoreHomedir(BASEDIR, "region", Bytes.toBytes("cf"));
 
   private static final byte[] KEY_A = Bytes.toBytes("aaa");
-  private static final byte[] KEY_B = Bytes.toBytes("bbb");
-  private static final byte[] KEY_C = Bytes.toBytes("ccc");
-  private static final byte[] KEY_D = Bytes.toBytes("ddd");
+  private static final byte[] KEY_B = Bytes.toBytes("aab");
+  private static final byte[] KEY_C = Bytes.toBytes("aac");
+  private static final byte[] KEY_D = Bytes.toBytes("aad");
 
   private static final KeyValue KV_A = new KeyValue(KEY_A, 0L);
   private static final KeyValue KV_B = new KeyValue(KEY_B, 0L);


[12/50] [abbrv] hbase git commit: HBASE-17857 Remove IS annotations from IA.Public classes

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshotException.java
index 05f3556..f6817e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshotException.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a snapshot could not be exported due to an error during the operation.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 @SuppressWarnings("serial")
 public class ExportSnapshotException extends HBaseSnapshotException {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
index 2fe58ed..bd185a1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 
 /**
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HBaseSnapshotException extends DoNotRetryIOException {
 
   private SnapshotDescription description;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
index 70e8d3b..de58077 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 
 /**
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RestoreSnapshotException extends HBaseSnapshotException {
   public RestoreSnapshotException(String msg, SnapshotDescription desc) {
     super(msg, desc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
index 2738b3d..9cfe83a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 
 /**
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class SnapshotCreationException extends HBaseSnapshotException {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
index e088408..ae574b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 
 
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class SnapshotDoesNotExistException extends HBaseSnapshotException {
   /**
    * @param msg full description of the failure

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
index 425f93a..9b31625 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 
 /**
@@ -26,7 +25,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class SnapshotExistsException extends HBaseSnapshotException {
   public SnapshotExistsException(String msg) {
     super(msg);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java
index b27ff65..343d702 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java
@@ -21,14 +21,12 @@ import java.io.IOException;
 
 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.util.Bytes;
 
 /**
  * Thrown if a table should be online/offline but is partially open
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class TablePartiallyOpenException extends IOException {
   private static final long serialVersionUID = 3571982660065058361L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/UnknownSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/UnknownSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/UnknownSnapshotException.java
index e4242f5..dc6dd56 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/UnknownSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/UnknownSnapshotException.java
@@ -18,14 +18,12 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Exception thrown when we get a request for a snapshot we don't recognize.
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class UnknownSnapshotException extends HBaseSnapshotException {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/util/FileSystemVersionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/FileSystemVersionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/FileSystemVersionException.java
index 2cd1d00..1613582 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/FileSystemVersionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/FileSystemVersionException.java
@@ -22,11 +22,9 @@ package org.apache.hadoop.hbase.util;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /** Thrown when the file system needs to be upgraded */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FileSystemVersionException extends IOException {
   private static final long serialVersionUID = 1004053363L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
index 2a9987c..dddd052 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
@@ -22,14 +22,12 @@ import java.io.IOException;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.codehaus.jackson.map.ObjectMapper;
 
 /**
  * Utility class for converting objects to JSON
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class JsonMapper {
   private JsonMapper() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
index 9acbb43..c5af6ff 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
@@ -40,26 +40,32 @@ import org.apache.hadoop.hbase.ClassFinder.Not;
 import org.apache.hadoop.hbase.ClassTestFinder.TestClassFilter;
 import org.apache.hadoop.hbase.ClassTestFinder.TestFileNameFilter;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
- * Test cases for ensuring our client visible classes have annotations
- * for {@link InterfaceAudience}.
- *
- * All classes in hbase-client and hbase-common module MUST have InterfaceAudience
- * annotations. All InterfaceAudience.Public annotated classes MUST also have InterfaceStability
- * annotations. Think twice about marking an interface InterfaceAudience.Public. Make sure that
- * it is an interface, not a class (for most cases), and clients will actually depend on it. Once
- * something is marked with Public, we cannot change the signatures within the major release. NOT
- * everything in the hbase-client module or every java public class has to be marked with
+ * Test cases for ensuring our client visible classes have annotations for
+ * {@link InterfaceAudience}.
+ * <p>
+ * All classes in hbase-client and hbase-common module MUST have InterfaceAudience annotations.
+ * Think twice about marking an interface InterfaceAudience.Public. Make sure that it is an
+ * interface, not a class (for most cases), and clients will actually depend on it. Once something
+ * is marked with Public, we cannot change the signatures within the major release. NOT everything
+ * in the hbase-client module or every java public class has to be marked with
  * InterfaceAudience.Public. ONLY the ones that an hbase application will directly use (Table, Get,
- * etc, versus ProtobufUtil).
- *
- * Also note that HBase has it's own annotations in hbase-annotations module with the same names
- * as in Hadoop. You should use the HBase's classes.
- *
- * See https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/InterfaceClassification.html
+ * etc, versus ProtobufUtil). And also, InterfaceAudience.Public annotated classes MUST NOT have
+ * InterfaceStability annotations. The stability of these classes only depends on versioning.
+ * <p>
+ * All classes which are marked as InterfaceAudience.LimitedPrivate MUST also have
+ * InterfaceStability annotations. The only exception is HBaseInterfaceAudience.CONFIG. It is used
+ * to indicate that the class name will be exposed in user facing configuration files.
+ * <p>
+ * Also note that HBase has it's own annotations in hbase-annotations module with the same names as
+ * in Hadoop. You should use the HBase's classes.
+ * <p>
+ * See
+ * https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/InterfaceClassification.html
  * and https://issues.apache.org/jira/browse/HBASE-10462.
  */
 @Category(SmallTests.class)
@@ -105,16 +111,15 @@ public class TestInterfaceAudienceAnnotations {
         return false;
       }
 
-      Class<?> ann = getAnnotation(c);
-      if (ann != null &&
-        !InterfaceAudience.Public.class.equals(ann)) {
+      Annotation ann = getAnnotation(c);
+      if (ann != null && !InterfaceAudience.Public.class.equals(ann.annotationType())) {
         return true;
       }
 
       return isAnnotatedPrivate(c.getEnclosingClass());
     }
 
-    protected Class<?> getAnnotation(Class<?> c) {
+    protected Annotation getAnnotation(Class<?> c) {
       // we should get only declared annotations, not inherited ones
       Annotation[] anns = c.getDeclaredAnnotations();
 
@@ -123,7 +128,7 @@ public class TestInterfaceAudienceAnnotations {
         // an enum instead we have three independent annotations!
         Class<?> type = ann.annotationType();
         if (isInterfaceAudienceClass(type)) {
-          return type;
+          return ann;
         }
       }
       return null;
@@ -159,13 +164,32 @@ public class TestInterfaceAudienceAnnotations {
     }
   }
 
-  /** Selects classes with one of the {@link InterfaceAudience.Public} annotation in their
-   * class declaration.
+  /**
+   * Selects classes with one of the {@link InterfaceAudience.Public} annotation in their class
+   * declaration.
    */
   class InterfaceAudiencePublicAnnotatedClassFilter extends InterfaceAudienceAnnotatedClassFilter {
     @Override
     public boolean isCandidateClass(Class<?> c) {
-      return (InterfaceAudience.Public.class.equals(getAnnotation(c)));
+      Annotation ann = getAnnotation(c);
+      return ann != null && InterfaceAudience.Public.class.equals(ann.annotationType());
+    }
+  }
+
+  /**
+   * Selects classes with one of the {@link InterfaceAudience.LimitedPrivate} annotation in their
+   * class declaration.
+   */
+  class InterfaceAudienceLimitedPrivateAnnotatedNotConfigClassFilter
+      extends InterfaceAudienceAnnotatedClassFilter {
+    @Override
+    public boolean isCandidateClass(Class<?> c) {
+      Annotation ann = getAnnotation(c);
+      if (ann == null || !InterfaceAudience.LimitedPrivate.class.equals(ann.annotationType())) {
+        return false;
+      }
+      InterfaceAudience.LimitedPrivate iaAnn = (InterfaceAudience.LimitedPrivate) ann;
+      return iaAnn.value().length == 0 || !HBaseInterfaceAudience.CONFIG.equals(iaAnn.value()[0]);
     }
   }
 
@@ -288,10 +312,11 @@ public class TestInterfaceAudienceAnnotations {
     );
 
     Set<Class<?>> classes = classFinder.findClasses(false);
-
-    LOG.info("These are the classes that DO NOT have @InterfaceAudience annotation:");
-    for (Class<?> clazz : classes) {
-      LOG.info(clazz);
+    if (!classes.isEmpty()) {
+      LOG.info("These are the classes that DO NOT have @InterfaceAudience annotation:");
+      for (Class<?> clazz : classes) {
+        LOG.info(clazz);
+      }
     }
 
     Assert.assertEquals("All classes should have @InterfaceAudience annotation",
@@ -300,10 +325,10 @@ public class TestInterfaceAudienceAnnotations {
 
   /**
    * Checks whether all the classes in client and common modules that are marked
-   * InterfaceAudience.Public also have {@link InterfaceStability} annotations.
+   * InterfaceAudience.Public do not have {@link InterfaceStability} annotations.
    */
   @Test
-  public void testInterfaceStabilityAnnotation()
+  public void testNoInterfaceStabilityAnnotationForPublicAPI()
       throws ClassNotFoundException, IOException, LinkageError {
 
     // find classes that are:
@@ -313,7 +338,7 @@ public class TestInterfaceAudienceAnnotations {
     // NOT test classes
     // AND NOT generated classes
     // AND are annotated with InterfaceAudience.Public
-    // AND NOT annotated with InterfaceStability
+    // AND annotated with InterfaceStability
     ClassFinder classFinder = new ClassFinder(
       new And(new MainCodeResourcePathFilter(),
               new TestFileNameFilter()),
@@ -324,18 +349,65 @@ public class TestInterfaceAudienceAnnotations {
               new Not(new ShadedProtobufClassFilter()),
               new InterfaceAudiencePublicAnnotatedClassFilter(),
               new Not(new IsInterfaceStabilityClassFilter()),
-              new Not(new InterfaceStabilityAnnotatedClassFilter()))
+              new InterfaceStabilityAnnotatedClassFilter())
     );
 
     Set<Class<?>> classes = classFinder.findClasses(false);
 
-    LOG.info("These are the classes that DO NOT have @InterfaceStability annotation:");
-    for (Class<?> clazz : classes) {
-      LOG.info(clazz);
+    if (!classes.isEmpty()) {
+      LOG.info("These are the @InterfaceAudience.Public classes that have @InterfaceStability " +
+          "annotation:");
+      for (Class<?> clazz : classes) {
+        LOG.info(clazz);
+      }
     }
 
-    Assert.assertEquals("All classes that are marked with @InterfaceAudience.Public should "
-        + "have @InterfaceStability annotation as well",
+    Assert.assertEquals("All classes that are marked with @InterfaceAudience.Public should not "
+        + "have @InterfaceStability annotation",
+      0, classes.size());
+  }
+
+  /**
+   * Checks whether all the classes in client and common modules that are marked
+   * InterfaceAudience.Public do not have {@link InterfaceStability} annotations.
+   */
+  @Ignore
+  @Test
+  public void testInterfaceStabilityAnnotationForLimitedAPI()
+      throws ClassNotFoundException, IOException, LinkageError {
+
+    // find classes that are:
+    // In the main jar
+    // AND are not in a hadoop-compat module
+    // AND are public
+    // NOT test classes
+    // AND NOT generated classes
+    // AND are annotated with InterfaceAudience.LimitedPrivate
+    // AND NOT annotated with InterfaceStability
+    ClassFinder classFinder = new ClassFinder(
+      new And(new MainCodeResourcePathFilter(),
+              new TestFileNameFilter()),
+      new Not((FileNameFilter)new TestFileNameFilter()),
+      new And(new PublicClassFilter(),
+              new Not(new TestClassFilter()),
+              new Not(new GeneratedClassFilter()),
+              new Not(new ShadedProtobufClassFilter()),
+              new InterfaceAudienceLimitedPrivateAnnotatedNotConfigClassFilter(),
+              new Not(new IsInterfaceStabilityClassFilter()),
+              new Not(new InterfaceStabilityAnnotatedClassFilter()))
+    );
+
+    Set<Class<?>> classes = classFinder.findClasses(false);
+
+    if (!classes.isEmpty()) {
+      LOG.info("These are the @InterfaceAudience.LimitedPrivate classes that DO NOT " +
+          "have @InterfaceStability annotation:");
+      for (Class<?> clazz : classes) {
+        LOG.info(clazz);
+      }
+    }
+    Assert.assertEquals("All classes that are marked with @InterfaceAudience.LimitedPrivate " +
+        "should have @InterfaceStability annotation",
       0, classes.size());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
index bbed218..d715d01 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
@@ -25,7 +25,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.DNS;
 import org.apache.hadoop.hbase.util.Strings;
@@ -68,7 +67,6 @@ import org.apache.hadoop.security.UserGroupInformation;
  * an example of configuring a user of this Auth Chore to run on a secure cluster.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class AuthUtil {
   private static final Log LOG = LogFactory.getLog(AuthUtil.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
index 53b319b..8a701f2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 
 /**
@@ -60,7 +59,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface Cell {
 
   //1) Row

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index bb5197f..6585173 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -36,7 +36,6 @@ import java.util.NavigableMap;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience.Private;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
 import org.apache.hadoop.hbase.io.util.Dictionary;
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.util.ClassSize;
  * method level.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class CellUtil {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
index 19363d0..70858f4 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
@@ -32,7 +32,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ScheduledChore.ChoreServicer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * ChoreService is a service that can be used to schedule instances of {@link ScheduledChore} to run
@@ -54,7 +53,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * Calling this method ensures that all scheduled chores are cancelled and cleaned up properly.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ChoreService implements ChoreServicer {
   private static final Log LOG = LogFactory.getLog(ChoreService.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
index 4baaabe..885219a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
@@ -26,7 +26,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 
@@ -34,7 +33,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig;
  * Adds HBase configuration files to a Configuration
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class HBaseConfiguration extends Configuration {
   private static final Log LOG = LogFactory.getLog(HBaseConfiguration.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
index edcbdc5..85e8725 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * All hbase specific IOExceptions should be subclasses of HBaseIOException
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class HBaseIOException extends IOException {
 
   private static final long serialVersionUID = 1L;
@@ -46,4 +44,4 @@ public class HBaseIOException extends IOException {
   public HBaseIOException(Throwable cause) {
       super(cause);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
index cb42e48..ae1db7c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseInterfaceAudience.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This class defines constants for different classes of hbase limited private apis
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class HBaseInterfaceAudience {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/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 3789f71..eff5690 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
@@ -29,14 +29,12 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * HConstants holds a bunch of HBase-related constants
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public final class HConstants {
   // NOTICE!!!! Please do not add a constants here, unless they are referenced by a lot of classes.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
index 23876ab..15f71a9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
@@ -27,7 +27,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * as opposed to a more tangible container.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class NamespaceDescriptor {
 
   /** System namespace name. */
@@ -162,7 +160,6 @@ public class NamespaceDescriptor {
   }
 
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public static class Builder {
     private String bName;
     private Map<String, String> bConfiguration = new TreeMap<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
index f35f27b..bb8bb08 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.NonceKey;
@@ -31,7 +30,6 @@ import org.apache.hadoop.util.StringUtils;
  * Procedure information
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ProcedureInfo implements Cloneable {
   private final long procId;
   private final String procName;
@@ -216,4 +214,4 @@ public class ProcedureInfo implements Cloneable {
     return procOwner.equals(user.getShortName());
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
index 306d285..5d95add 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * POJO representing Procedure State
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum ProcedureState {
   INITIALIZING, RUNNABLE, WAITING, WAITING_TIMEOUT, ROLLEDBACK, FINISHED;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
index 422ca1a..2d1eec5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
@@ -24,7 +24,6 @@ import java.util.concurrent.TimeUnit;
 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 com.google.common.annotations.VisibleForTesting;
 
@@ -42,7 +41,6 @@ import com.google.common.annotations.VisibleForTesting;
  * an entry being added to a queue, etc.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class ScheduledChore implements Runnable {
   private static final Log LOG = LogFactory.getLog(ScheduledChore.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
index 0c0a7ff..fabf0c0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
@@ -25,7 +25,6 @@ import java.util.Locale;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +55,6 @@ import com.google.common.net.InetAddresses;
  * <p>Immutable.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ServerName implements Comparable<ServerName>, Serializable {
   private static final long serialVersionUID = 1367463982557264981L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java
index 9adaa1a..cdb802c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Implementers are Stoppable.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public interface Stoppable {
   /**
    * Stop this service.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
index cba03c0..c4c15d0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
@@ -24,7 +24,6 @@ import java.util.Set;
 import java.util.concurrent.CopyOnWriteArraySet;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 
@@ -53,7 +52,6 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class TableName implements Comparable<TableName> {
 
   /** See {@link #createTableNameIfNecessary(ByteBuffer, ByteBuffer)} */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
index 2133750..8af562e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.filter;
 import java.nio.ByteBuffer;
 
 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.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 
 /** Base class for byte array comparators */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 // TODO Now we are deviating a lot from the actual Comparable<byte[]> what this implements, by
 // adding special compareTo methods. We have to clean it. Deprecate this class and replace it
 // with a more generic one which says it compares bytes (not necessary a byte array only)

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
index f6f7def..9aaa431 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
@@ -28,7 +28,6 @@ import java.nio.channels.Channels;
 import java.nio.channels.WritableByteChannel;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * Not thread safe!
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ByteBufferOutputStream extends OutputStream
     implements ByteBufferWriter {
   

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java
index f658210..5c7c292 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java
@@ -25,7 +25,6 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
@@ -40,7 +39,6 @@ import org.apache.hadoop.io.WritableComparator;
  * buffer is accessed when we go to serialize.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
     value="EQ_CHECK_FOR_OPERAND_NOT_COMPATIBLE_WITH_THIS",
     justification="It has been like this forever")
@@ -229,7 +227,6 @@ implements WritableComparable<ImmutableBytesWritable> {
   /** A Comparator optimized for ImmutableBytesWritable.
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public static class Comparator extends WritableComparator {
     private BytesWritable.Comparator comparator =
       new BytesWritable.Comparator();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
index 77b9495..764b2a0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.io;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -34,7 +33,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  *<p>Immutable. Thread-safe.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TimeRange {
   public static final long INITIAL_MIN_TIMESTAMP = 0L;
   public static final long INITIAL_MAX_TIMESTAMP = Long.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
index 8dfab44..6f63f80 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
@@ -28,7 +28,6 @@ import org.apache.commons.logging.LogFactory;
 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;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -101,7 +100,6 @@ public final class Compression {
       value="SE_TRANSIENT_FIELD_NOT_RESTORED",
       justification="We are not serializing so doesn't apply (not sure why transient though)")
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public static enum Algorithm {
     LZO("lzo") {
       // Use base type to avoid compile-time dependencies.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Cipher.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Cipher.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Cipher.java
index e19a13d..3f4bf2b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Cipher.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Cipher.java
@@ -22,13 +22,11 @@ import java.io.OutputStream;
 import java.security.Key;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A common interface for a cryptographic algorithm.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class Cipher {
 
   public static final int KEY_LENGTH = 16;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CipherProvider.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CipherProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CipherProvider.java
index 5a475cc..e457c13 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CipherProvider.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CipherProvider.java
@@ -18,14 +18,12 @@ package org.apache.hadoop.hbase.io.crypto;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * An CipherProvider contributes support for various cryptographic
  * Ciphers.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface CipherProvider extends Configurable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Context.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Context.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Context.java
index 1e2881e..a8dc396 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Context.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Context.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.MD5Hash;
 
 import com.google.common.base.Preconditions;
@@ -31,7 +30,6 @@ import com.google.common.base.Preconditions;
  * Crypto context. Encapsulates an encryption algorithm and its key material.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class Context implements Configurable {
   private Configuration conf;
   private Cipher cipher;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CryptoCipherProvider.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CryptoCipherProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CryptoCipherProvider.java
index 3f5cd2d..6f78e83 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CryptoCipherProvider.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/CryptoCipherProvider.java
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.io.crypto;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.crypto.aes.CommonsCryptoAES;
 
 /**
  * The default cipher provider. Supports AES via the Commons Crypto.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class CryptoCipherProvider implements CipherProvider {
 
   private static CryptoCipherProvider instance;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Decryptor.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Decryptor.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Decryptor.java
index d3029db..947e11a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Decryptor.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Decryptor.java
@@ -21,13 +21,11 @@ import java.io.InputStream;
 import java.security.Key;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Decryptors apply a cipher to an InputStream to recover plaintext.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface Decryptor {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/DefaultCipherProvider.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/DefaultCipherProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/DefaultCipherProvider.java
index 4f2aebe..9c82b2a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/DefaultCipherProvider.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/DefaultCipherProvider.java
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.io.crypto;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.crypto.aes.AES;
 
 /**
  * The default cipher provider. Supports AES via the JCE.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class DefaultCipherProvider implements CipherProvider {
 
   private static DefaultCipherProvider instance;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java
index b6c2e97..e8727a7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 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.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -48,7 +47,6 @@ import org.apache.hadoop.util.ReflectionUtils;
  * A facade for encryption algorithms and related support.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public final class Encryption {
 
   private static final Log LOG = LogFactory.getLog(Encryption.class);
@@ -57,7 +55,6 @@ public final class Encryption {
    * Crypto context
    */
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public static class Context extends org.apache.hadoop.hbase.io.crypto.Context {
 
     /** The null crypto context */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryptor.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryptor.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryptor.java
index cda703d..4e84a68 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryptor.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryptor.java
@@ -21,13 +21,11 @@ import java.io.OutputStream;
 import java.security.Key;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Encryptors apply a cipher to an OutputStream to produce ciphertext.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface Encryptor {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyProvider.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyProvider.java
index 515a664..e0542e1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyProvider.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyProvider.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.io.crypto;
 import java.security.Key;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * KeyProvider is a interface to abstract the different methods of retrieving
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  *
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface KeyProvider {
 
   public static final String PASSWORD = "password";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
index 7e3c013..1f6c83a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
@@ -34,7 +34,6 @@ import java.util.Locale;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * A basic KeyProvider that can resolve keys from a protected KeyStore file
@@ -71,7 +70,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * LoadStoreParameters.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class KeyStoreKeyProvider implements KeyProvider {
 
   protected KeyStore store;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java
index d7535e5..1133b91 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * in the HBase mailing list to prevent collisions.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum DataBlockEncoding {
 
   /** Disable data block encoding. */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
index 4cc636e..3ee8cfc 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.net;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import com.google.common.net.HostAndPort;
 
@@ -30,7 +29,6 @@ import com.google.common.net.HostAndPort;
  * <p>In implementation this class is a facade over Guava's {@link HostAndPort}.
  * We cannot have Guava classes in our API hence this Type.
  */
-@InterfaceStability.Evolving
 @InterfaceAudience.Public
 public class Address implements Comparable<Address> {
   private HostAndPort hostAndPort;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
index 9219c23..c3a6dc7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
@@ -25,14 +25,12 @@ import java.util.TreeSet;
 
 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.net.Address;
 
 /**
  * Stores the group information of region server groups.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RSGroupInfo {
   public static final String DEFAULT_GROUP = "default";
   public static final String NAMESPACE_DESC_PROP_GROUP = "hbase.rsgroup.name";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
index be2a0d3..c7d65be 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
@@ -31,7 +31,6 @@ import java.util.concurrent.ExecutionException;
 import com.google.common.cache.LoadingCache;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Methods;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.SecurityUtil;
@@ -51,7 +50,6 @@ import org.apache.hadoop.security.token.TokenIdentifier;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public abstract class User {
   public static final String HBASE_SECURITY_CONF_KEY =
       "hbase.security.authentication";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java
index cd43448..08f1112 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/DataType.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface DataType<T> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java
index 1caf518..cf662e7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/FixedLengthWrapper.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange;
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange;
  * variant.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FixedLengthWrapper<T> implements DataType<T> {
 
   protected final DataType<T> base;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java
index cd6f614..c40964f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlob.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeBlobCopy(PositionedByteRange, byte[], int, int, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedBlob extends OrderedBytesBase<byte[]> {
 
   public static final OrderedBlob ASCENDING = new OrderedBlob(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java
index 22a321b..a4ecb9a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBlobVar.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeBlobVar(PositionedByteRange, byte[], int, int, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedBlobVar extends OrderedBytesBase<byte[]> {
 
   public static final OrderedBlobVar ASCENDING = new OrderedBlobVar(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java
index 682202d..fcc823d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedBytesBase.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * implementations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class OrderedBytesBase<T> implements DataType<T> {
 
   protected final Order order;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java
index a417f77..f1af97e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat32.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeFloat32(PositionedByteRange, float, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedFloat32 extends OrderedBytesBase<Float> {
 
   public static final OrderedFloat32 ASCENDING = new OrderedFloat32(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java
index 891b07c..a8036cd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedFloat64.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeFloat64(PositionedByteRange, double, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedFloat64 extends OrderedBytesBase<Double> {
 
   public static final OrderedFloat64 ASCENDING = new OrderedFloat64(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt16.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt16.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt16.java
index b968f5d..6930c86 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt16.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt16.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeInt16(PositionedByteRange, short, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedInt16 extends OrderedBytesBase<Short> {
 
   public static final OrderedInt16 ASCENDING = new OrderedInt16(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java
index a504f13..828bae9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt32.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeInt32(PositionedByteRange, int, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedInt32 extends OrderedBytesBase<Integer> {
 
   public static final OrderedInt32 ASCENDING = new OrderedInt32(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java
index 3fb65e3..219911e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt64.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeInt64(PositionedByteRange, long, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedInt64 extends OrderedBytesBase<Long> {
 
   public static final OrderedInt64 ASCENDING = new OrderedInt64(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt8.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt8.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt8.java
index 50fcec0..3767b75 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt8.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedInt8.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeInt8(PositionedByteRange, byte, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedInt8 extends OrderedBytesBase<Byte> {
 
   public static final OrderedInt8 ASCENDING = new OrderedInt8(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java
index db34cf5..9a37e71 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedNumeric.java
@@ -21,7 +21,6 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.util.SimplePositionedMutableByteRange;
  * from text. Built on {@link OrderedBytes#encodeNumeric(PositionedByteRange, BigDecimal, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedNumeric extends OrderedBytesBase<Number> {
 
   public static final OrderedNumeric ASCENDING = new OrderedNumeric(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java
index f7401bf..e5fcd5a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/OrderedString.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.OrderedBytes;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * {@link OrderedBytes#encodeString(PositionedByteRange, String, Order)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class OrderedString extends OrderedBytesBase<String> {
 
   public static final OrderedString ASCENDING = new OrderedString(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
index 3d545f6..2718e98 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -31,7 +30,6 @@ import com.google.protobuf.Message;
  * {@code PBKeyValue} in {@code hbase-examples} module.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class PBType<T extends Message> implements DataType<T> {
   @Override
   public boolean isOrderPreserving() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawByte.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawByte.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawByte.java
index 9f90350..fdaff10 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawByte.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawByte.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see Bytes#putByte(byte[], int, byte)
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawByte implements DataType<Byte> {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java
index bea3c5f..449edfc 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytes.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see OrderedBlobVar
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawBytes implements DataType<byte[]> {
 
   public static final RawBytes ASCENDING = new RawBytes(Order.ASCENDING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java
index bfd6416..0333721 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesFixedLength.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -34,7 +33,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see OrderedBlobVar
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawBytesFixedLength extends FixedLengthWrapper<byte[]> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java
index 8bc4c20..1b67bd8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawBytesTerminated.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
 
@@ -32,7 +31,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see OrderedBlob
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawBytesTerminated extends TerminatedWrapper<byte[]> {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java
index 776639c..c100ccd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawDouble.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see Bytes#toDouble(byte[])
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawDouble implements DataType<Double> {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java
index 5c2f823..751e4ef 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawFloat.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see Bytes#toFloat(byte[])
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawFloat implements DataType<Float> {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java
index 0d9e4eb..ab09322 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawInteger.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see Bytes#toInt(byte[])
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawInteger implements DataType<Integer> {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java
index b8bbcd2..0da3bbd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawLong.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see Bytes#toLong(byte[])
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawLong implements DataType<Long> {
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawShort.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawShort.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawShort.java
index bc1ef30..cb342e7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawShort.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/RawShort.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.types;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Order;
 import org.apache.hadoop.hbase.util.PositionedByteRange;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
  * @see Bytes#toShort(byte[])
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RawShort implements DataType<Short> {
 
   @Override


[05/50] [abbrv] hbase git commit: HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Option.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Option.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Option.java
index 86f8d4b..cfd28c9 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Option.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Option.java
@@ -94,10 +94,13 @@ public  final class Option extends
   private volatile java.lang.Object name_;
   /**
    * <pre>
-   * The option's name. For example, `"java_package"`.
+   * The option's name. For protobuf built-in options (options defined in
+   * descriptor.proto), this is the short name. For example, `"map_entry"`.
+   * For custom options, it should be the fully-qualified name. For example,
+   * `"google.api.http"`.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -113,10 +116,13 @@ public  final class Option extends
   }
   /**
    * <pre>
-   * The option's name. For example, `"java_package"`.
+   * The option's name. For protobuf built-in options (options defined in
+   * descriptor.proto), this is the short name. For example, `"map_entry"`.
+   * For custom options, it should be the fully-qualified name. For example,
+   * `"google.api.http"`.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -136,30 +142,39 @@ public  final class Option extends
   private org.apache.hadoop.hbase.shaded.com.google.protobuf.Any value_;
   /**
    * <pre>
-   * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+   * The option's value packed in an Any message. If the value is a primitive,
+   * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+   * should be used. If the value is an enum, it should be stored as an int32
+   * value using the google.protobuf.Int32Value type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Any value = 2;</code>
+   * <code>.google.protobuf.Any value = 2;</code>
    */
   public boolean hasValue() {
     return value_ != null;
   }
   /**
    * <pre>
-   * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+   * The option's value packed in an Any message. If the value is a primitive,
+   * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+   * should be used. If the value is an enum, it should be stored as an int32
+   * value using the google.protobuf.Int32Value type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Any value = 2;</code>
+   * <code>.google.protobuf.Any value = 2;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Any getValue() {
     return value_ == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.Any.getDefaultInstance() : value_;
   }
   /**
    * <pre>
-   * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+   * The option's value packed in an Any message. If the value is a primitive,
+   * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+   * should be used. If the value is an enum, it should be stored as an int32
+   * value using the google.protobuf.Int32Value type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Any value = 2;</code>
+   * <code>.google.protobuf.Any value = 2;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.AnyOrBuilder getValueOrBuilder() {
     return getValue();
@@ -229,7 +244,7 @@ public  final class Option extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + NAME_FIELD_NUMBER;
     hash = (53 * hash) + getName().hashCode();
     if (hasValue()) {
@@ -472,10 +487,13 @@ public  final class Option extends
     private java.lang.Object name_ = "";
     /**
      * <pre>
-     * The option's name. For example, `"java_package"`.
+     * The option's name. For protobuf built-in options (options defined in
+     * descriptor.proto), this is the short name. For example, `"map_entry"`.
+     * For custom options, it should be the fully-qualified name. For example,
+     * `"google.api.http"`.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -491,10 +509,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's name. For example, `"java_package"`.
+     * The option's name. For protobuf built-in options (options defined in
+     * descriptor.proto), this is the short name. For example, `"map_entry"`.
+     * For custom options, it should be the fully-qualified name. For example,
+     * `"google.api.http"`.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -511,10 +532,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's name. For example, `"java_package"`.
+     * The option's name. For protobuf built-in options (options defined in
+     * descriptor.proto), this is the short name. For example, `"map_entry"`.
+     * For custom options, it should be the fully-qualified name. For example,
+     * `"google.api.http"`.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -528,10 +552,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's name. For example, `"java_package"`.
+     * The option's name. For protobuf built-in options (options defined in
+     * descriptor.proto), this is the short name. For example, `"map_entry"`.
+     * For custom options, it should be the fully-qualified name. For example,
+     * `"google.api.http"`.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder clearName() {
       
@@ -541,10 +568,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's name. For example, `"java_package"`.
+     * The option's name. For protobuf built-in options (options defined in
+     * descriptor.proto), this is the short name. For example, `"map_entry"`.
+     * For custom options, it should be the fully-qualified name. For example,
+     * `"google.api.http"`.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -563,20 +593,26 @@ public  final class Option extends
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Any, org.apache.hadoop.hbase.shaded.com.google.protobuf.Any.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.AnyOrBuilder> valueBuilder_;
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public boolean hasValue() {
       return valueBuilder_ != null || value_ != null;
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Any getValue() {
       if (valueBuilder_ == null) {
@@ -587,10 +623,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.Any value) {
       if (valueBuilder_ == null) {
@@ -607,10 +646,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public Builder setValue(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Any.Builder builderForValue) {
@@ -625,10 +667,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public Builder mergeValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.Any value) {
       if (valueBuilder_ == null) {
@@ -647,10 +692,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public Builder clearValue() {
       if (valueBuilder_ == null) {
@@ -665,10 +713,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Any.Builder getValueBuilder() {
       
@@ -677,10 +728,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.AnyOrBuilder getValueOrBuilder() {
       if (valueBuilder_ != null) {
@@ -692,10 +746,13 @@ public  final class Option extends
     }
     /**
      * <pre>
-     * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+     * The option's value packed in an Any message. If the value is a primitive,
+     * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+     * should be used. If the value is an enum, it should be stored as an int32
+     * value using the google.protobuf.Int32Value type.
      * </pre>
      *
-     * <code>optional .google.protobuf.Any value = 2;</code>
+     * <code>.google.protobuf.Any value = 2;</code>
      */
     private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Any, org.apache.hadoop.hbase.shaded.com.google.protobuf.Any.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.AnyOrBuilder> 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/OptionOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/OptionOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/OptionOrBuilder.java
index 20f2b8c..3fea10a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/OptionOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/OptionOrBuilder.java
@@ -9,44 +9,59 @@ public interface OptionOrBuilder extends
 
   /**
    * <pre>
-   * The option's name. For example, `"java_package"`.
+   * The option's name. For protobuf built-in options (options defined in
+   * descriptor.proto), this is the short name. For example, `"map_entry"`.
+   * For custom options, it should be the fully-qualified name. For example,
+   * `"google.api.http"`.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   java.lang.String getName();
   /**
    * <pre>
-   * The option's name. For example, `"java_package"`.
+   * The option's name. For protobuf built-in options (options defined in
+   * descriptor.proto), this is the short name. For example, `"map_entry"`.
+   * For custom options, it should be the fully-qualified name. For example,
+   * `"google.api.http"`.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
 
   /**
    * <pre>
-   * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+   * The option's value packed in an Any message. If the value is a primitive,
+   * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+   * should be used. If the value is an enum, it should be stored as an int32
+   * value using the google.protobuf.Int32Value type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Any value = 2;</code>
+   * <code>.google.protobuf.Any value = 2;</code>
    */
   boolean hasValue();
   /**
    * <pre>
-   * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+   * The option's value packed in an Any message. If the value is a primitive,
+   * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+   * should be used. If the value is an enum, it should be stored as an int32
+   * value using the google.protobuf.Int32Value type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Any value = 2;</code>
+   * <code>.google.protobuf.Any value = 2;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Any getValue();
   /**
    * <pre>
-   * The option's value. For example, `"org.apache.hadoop.hbase.shaded.com.google.protobuf"`.
+   * The option's value packed in an Any message. If the value is a primitive,
+   * the corresponding wrapper type defined in google/protobuf/wrappers.proto
+   * should be used. If the value is an enum, it should be stored as an int32
+   * value using the google.protobuf.Int32Value type.
    * </pre>
    *
-   * <code>optional .google.protobuf.Any value = 2;</code>
+   * <code>.google.protobuf.Any value = 2;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.AnyOrBuilder getValueOrBuilder();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SmallSortedMap.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SmallSortedMap.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SmallSortedMap.java
index 2dfabd3..a24c2ce 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SmallSortedMap.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SmallSortedMap.java
@@ -197,6 +197,7 @@ class SmallSortedMap<K extends Comparable<K>, V> extends AbstractMap<K, V> {
         overflowEntries.entrySet();
   }
 
+
   @Override
   public int size() {
     return entryList.size() + overflowEntries.size();
@@ -356,6 +357,7 @@ class SmallSortedMap<K extends Comparable<K>, V> extends AbstractMap<K, V> {
     return lazyEntrySet;
   }
 
+
   /**
    * @throws UnsupportedOperationException if {@link #makeImmutable()} has
    *         has been called.
@@ -525,6 +527,7 @@ class SmallSortedMap<K extends Comparable<K>, V> extends AbstractMap<K, V> {
     }
   }
 
+
   /**
    * Iterator implementation that switches from the entry array to the overflow
    * entries appropriately.
@@ -617,43 +620,43 @@ class SmallSortedMap<K extends Comparable<K>, V> extends AbstractMap<K, V> {
       return (Iterable<T>) ITERABLE;
     }
   }
-  
+
   @Override
   public boolean equals(Object o) {
     if (this == o) {
       return true;
     }
-    
+
     if (!(o instanceof SmallSortedMap)) {
       return super.equals(o);
     }
-    
+
     SmallSortedMap<?, ?> other = (SmallSortedMap<?, ?>) o;
     final int size = size();
     if (size != other.size()) {
       return false;
     }
-    
+
     // Best effort try to avoid allocating an entry set.
     final int numArrayEntries = getNumArrayEntries();
     if (numArrayEntries != other.getNumArrayEntries()) {
       return entrySet().equals(other.entrySet());
     }
-    
+
     for (int i = 0; i < numArrayEntries; i++) {
       if (!getArrayEntryAt(i).equals(other.getArrayEntryAt(i))) {
         return false;
       }
     }
-    
+
     if (numArrayEntries != size) {
       return overflowEntries.equals(other.overflowEntries);
     }
-    
-    
+
+
     return true;
   }
-  
+
   @Override
   public int hashCode() {
     int h = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContext.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContext.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContext.java
index 720fd63..ab4938c 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContext.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContext.java
@@ -85,7 +85,7 @@ public  final class SourceContext extends
    * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
    * </pre>
    *
-   * <code>optional string file_name = 1;</code>
+   * <code>string file_name = 1;</code>
    */
   public java.lang.String getFileName() {
     java.lang.Object ref = fileName_;
@@ -105,7 +105,7 @@ public  final class SourceContext extends
    * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
    * </pre>
    *
-   * <code>optional string file_name = 1;</code>
+   * <code>string file_name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getFileNameBytes() {
@@ -173,7 +173,7 @@ public  final class SourceContext extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + FILE_NAME_FIELD_NUMBER;
     hash = (53 * hash) + getFileName().hashCode();
     hash = (29 * hash) + unknownFields.hashCode();
@@ -402,7 +402,7 @@ public  final class SourceContext extends
      * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
      * </pre>
      *
-     * <code>optional string file_name = 1;</code>
+     * <code>string file_name = 1;</code>
      */
     public java.lang.String getFileName() {
       java.lang.Object ref = fileName_;
@@ -422,7 +422,7 @@ public  final class SourceContext extends
      * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
      * </pre>
      *
-     * <code>optional string file_name = 1;</code>
+     * <code>string file_name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getFileNameBytes() {
@@ -443,7 +443,7 @@ public  final class SourceContext extends
      * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
      * </pre>
      *
-     * <code>optional string file_name = 1;</code>
+     * <code>string file_name = 1;</code>
      */
     public Builder setFileName(
         java.lang.String value) {
@@ -461,7 +461,7 @@ public  final class SourceContext extends
      * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
      * </pre>
      *
-     * <code>optional string file_name = 1;</code>
+     * <code>string file_name = 1;</code>
      */
     public Builder clearFileName() {
       
@@ -475,7 +475,7 @@ public  final class SourceContext extends
      * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
      * </pre>
      *
-     * <code>optional string file_name = 1;</code>
+     * <code>string file_name = 1;</code>
      */
     public Builder setFileNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextOrBuilder.java
index 4a59bb5..c35eceb 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextOrBuilder.java
@@ -13,7 +13,7 @@ public interface SourceContextOrBuilder extends
    * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
    * </pre>
    *
-   * <code>optional string file_name = 1;</code>
+   * <code>string file_name = 1;</code>
    */
   java.lang.String getFileName();
   /**
@@ -22,7 +22,7 @@ public interface SourceContextOrBuilder extends
    * protobuf element.  For example: `"google/protobuf/source_context.proto"`.
    * </pre>
    *
-   * <code>optional string file_name = 1;</code>
+   * <code>string file_name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getFileNameBytes();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextProto.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextProto.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextProto.java
index 6ee1bda..48f2e3f 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextProto.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/SourceContextProto.java
@@ -30,9 +30,11 @@ public final class SourceContextProto {
     java.lang.String[] descriptorData = {
       "\n$google/protobuf/source_context.proto\022\017" +
       "google.protobuf\"\"\n\rSourceContext\022\021\n\tfile" +
-      "_name\030\001 \001(\tBR\n\023com.google.protobufB\022Sour" +
-      "ceContextProtoP\001\242\002\003GPB\252\002\036Google.Protobuf" +
-      ".WellKnownTypesb\006proto3"
+      "_name\030\001 \001(\tB\225\001\n\023com.google.protobufB\022Sou" +
+      "rceContextProtoP\001ZAgoogle.golang.org/gen" +
+      "proto/protobuf/source_context;source_con" +
+      "text\242\002\003GPB\252\002\036Google.Protobuf.WellKnownTy" +
+      "pesb\006proto3"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValue.java
index 70f4acf..f3e2ef6 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValue.java
@@ -84,7 +84,7 @@ public  final class StringValue extends
    * The string value.
    * </pre>
    *
-   * <code>optional string value = 1;</code>
+   * <code>string value = 1;</code>
    */
   public java.lang.String getValue() {
     java.lang.Object ref = value_;
@@ -103,7 +103,7 @@ public  final class StringValue extends
    * The string value.
    * </pre>
    *
-   * <code>optional string value = 1;</code>
+   * <code>string value = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getValueBytes() {
@@ -171,7 +171,7 @@ public  final class StringValue extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + getValue().hashCode();
     hash = (29 * hash) + unknownFields.hashCode();
@@ -399,7 +399,7 @@ public  final class StringValue extends
      * The string value.
      * </pre>
      *
-     * <code>optional string value = 1;</code>
+     * <code>string value = 1;</code>
      */
     public java.lang.String getValue() {
       java.lang.Object ref = value_;
@@ -418,7 +418,7 @@ public  final class StringValue extends
      * The string value.
      * </pre>
      *
-     * <code>optional string value = 1;</code>
+     * <code>string value = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getValueBytes() {
@@ -438,7 +438,7 @@ public  final class StringValue extends
      * The string value.
      * </pre>
      *
-     * <code>optional string value = 1;</code>
+     * <code>string value = 1;</code>
      */
     public Builder setValue(
         java.lang.String value) {
@@ -455,7 +455,7 @@ public  final class StringValue extends
      * The string value.
      * </pre>
      *
-     * <code>optional string value = 1;</code>
+     * <code>string value = 1;</code>
      */
     public Builder clearValue() {
       
@@ -468,7 +468,7 @@ public  final class StringValue extends
      * The string value.
      * </pre>
      *
-     * <code>optional string value = 1;</code>
+     * <code>string value = 1;</code>
      */
     public Builder setValueBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValueOrBuilder.java
index 81873cd..7ac0ee8 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/StringValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface StringValueOrBuilder extends
    * The string value.
    * </pre>
    *
-   * <code>optional string value = 1;</code>
+   * <code>string value = 1;</code>
    */
   java.lang.String getValue();
   /**
@@ -20,7 +20,7 @@ public interface StringValueOrBuilder extends
    * The string value.
    * </pre>
    *
-   * <code>optional string value = 1;</code>
+   * <code>string value = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getValueBytes();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Struct.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Struct.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Struct.java
index ced7eea..2607900 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Struct.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Struct.java
@@ -253,7 +253,7 @@ public  final class Struct extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     if (!internalGetFields().getMap().isEmpty()) {
       hash = (37 * hash) + FIELDS_FIELD_NUMBER;
       hash = (53 * hash) + internalGetFields().hashCode();
@@ -597,7 +597,8 @@ public  final class Struct extends
     }
 
     public Builder clearFields() {
-      getMutableFields().clear();
+      internalGetMutableFields().getMutableMap()
+          .clear();
       return this;
     }
     /**
@@ -611,7 +612,8 @@ public  final class Struct extends
     public Builder removeFields(
         java.lang.String key) {
       if (key == null) { throw new java.lang.NullPointerException(); }
-      getMutableFields().remove(key);
+      internalGetMutableFields().getMutableMap()
+          .remove(key);
       return this;
     }
     /**
@@ -634,7 +636,8 @@ public  final class Struct extends
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Value value) {
       if (key == null) { throw new java.lang.NullPointerException(); }
       if (value == null) { throw new java.lang.NullPointerException(); }
-      getMutableFields().put(key, value);
+      internalGetMutableFields().getMutableMap()
+          .put(key, value);
       return this;
     }
     /**
@@ -647,7 +650,8 @@ public  final class Struct extends
 
     public Builder putAllFields(
         java.util.Map<java.lang.String, org.apache.hadoop.hbase.shaded.com.google.protobuf.Value> values) {
-      getMutableFields().putAll(values);
+      internalGetMutableFields().getMutableMap()
+          .putAll(values);
       return this;
     }
     public final Builder setUnknownFields(

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Syntax.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Syntax.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Syntax.java
index 0e65a5d..f03d70b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Syntax.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Syntax.java
@@ -95,8 +95,7 @@ public enum Syntax
   }
   public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
       getDescriptor() {
-    return org.apache.hadoop.hbase.shaded.com.google.protobuf.TypeProto.getDescriptor()
-        .getEnumTypes().get(0);
+    return org.apache.hadoop.hbase.shaded.com.google.protobuf.TypeProto.getDescriptor().getEnumTypes().get(0);
   }
 
   private static final Syntax[] VALUES = values();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Timestamp.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Timestamp.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Timestamp.java
index 29c61c3..0023d50 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Timestamp.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Timestamp.java
@@ -126,7 +126,7 @@ public  final class Timestamp extends
    * 9999-12-31T23:59:59Z inclusive.
    * </pre>
    *
-   * <code>optional int64 seconds = 1;</code>
+   * <code>int64 seconds = 1;</code>
    */
   public long getSeconds() {
     return seconds_;
@@ -142,7 +142,7 @@ public  final class Timestamp extends
    * inclusive.
    * </pre>
    *
-   * <code>optional int32 nanos = 2;</code>
+   * <code>int32 nanos = 2;</code>
    */
   public int getNanos() {
     return nanos_;
@@ -210,7 +210,7 @@ public  final class Timestamp extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + SECONDS_FIELD_NUMBER;
     hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
         getSeconds());
@@ -483,7 +483,7 @@ public  final class Timestamp extends
      * 9999-12-31T23:59:59Z inclusive.
      * </pre>
      *
-     * <code>optional int64 seconds = 1;</code>
+     * <code>int64 seconds = 1;</code>
      */
     public long getSeconds() {
       return seconds_;
@@ -495,7 +495,7 @@ public  final class Timestamp extends
      * 9999-12-31T23:59:59Z inclusive.
      * </pre>
      *
-     * <code>optional int64 seconds = 1;</code>
+     * <code>int64 seconds = 1;</code>
      */
     public Builder setSeconds(long value) {
       
@@ -510,7 +510,7 @@ public  final class Timestamp extends
      * 9999-12-31T23:59:59Z inclusive.
      * </pre>
      *
-     * <code>optional int64 seconds = 1;</code>
+     * <code>int64 seconds = 1;</code>
      */
     public Builder clearSeconds() {
       
@@ -528,7 +528,7 @@ public  final class Timestamp extends
      * inclusive.
      * </pre>
      *
-     * <code>optional int32 nanos = 2;</code>
+     * <code>int32 nanos = 2;</code>
      */
     public int getNanos() {
       return nanos_;
@@ -541,7 +541,7 @@ public  final class Timestamp extends
      * inclusive.
      * </pre>
      *
-     * <code>optional int32 nanos = 2;</code>
+     * <code>int32 nanos = 2;</code>
      */
     public Builder setNanos(int value) {
       
@@ -557,7 +557,7 @@ public  final class Timestamp extends
      * inclusive.
      * </pre>
      *
-     * <code>optional int32 nanos = 2;</code>
+     * <code>int32 nanos = 2;</code>
      */
     public Builder clearNanos() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TimestampOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TimestampOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TimestampOrBuilder.java
index 6ec1b04..fca3c16 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TimestampOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TimestampOrBuilder.java
@@ -14,7 +14,7 @@ public interface TimestampOrBuilder extends
    * 9999-12-31T23:59:59Z inclusive.
    * </pre>
    *
-   * <code>optional int64 seconds = 1;</code>
+   * <code>int64 seconds = 1;</code>
    */
   long getSeconds();
 
@@ -26,7 +26,7 @@ public interface TimestampOrBuilder extends
    * inclusive.
    * </pre>
    *
-   * <code>optional int32 nanos = 2;</code>
+   * <code>int32 nanos = 2;</code>
    */
   int getNanos();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Type.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Type.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Type.java
index 3a0ea49..f46f351 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Type.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Type.java
@@ -143,7 +143,7 @@ public  final class Type extends
    * The fully qualified message name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -162,7 +162,7 @@ public  final class Type extends
    * The fully qualified message name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -340,7 +340,7 @@ public  final class Type extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   public boolean hasSourceContext() {
     return sourceContext_ != null;
@@ -350,7 +350,7 @@ public  final class Type extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext() {
     return sourceContext_ == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.getDefaultInstance() : sourceContext_;
@@ -360,7 +360,7 @@ public  final class Type extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder() {
     return getSourceContext();
@@ -373,7 +373,7 @@ public  final class Type extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+   * <code>.google.protobuf.Syntax syntax = 6;</code>
    */
   public int getSyntaxValue() {
     return syntax_;
@@ -383,7 +383,7 @@ public  final class Type extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+   * <code>.google.protobuf.Syntax syntax = 6;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -493,7 +493,7 @@ public  final class Type extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + NAME_FIELD_NUMBER;
     hash = (53 * hash) + getName().hashCode();
     if (getFieldsCount() > 0) {
@@ -863,7 +863,7 @@ public  final class Type extends
      * The fully qualified message name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -882,7 +882,7 @@ public  final class Type extends
      * The fully qualified message name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -902,7 +902,7 @@ public  final class Type extends
      * The fully qualified message name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -919,7 +919,7 @@ public  final class Type extends
      * The fully qualified message name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder clearName() {
       
@@ -932,7 +932,7 @@ public  final class Type extends
      * The fully qualified message name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -1708,7 +1708,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public boolean hasSourceContext() {
       return sourceContextBuilder_ != null || sourceContext_ != null;
@@ -1718,7 +1718,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext() {
       if (sourceContextBuilder_ == null) {
@@ -1732,7 +1732,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder setSourceContext(org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext value) {
       if (sourceContextBuilder_ == null) {
@@ -1752,7 +1752,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder setSourceContext(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder builderForValue) {
@@ -1770,7 +1770,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder mergeSourceContext(org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext value) {
       if (sourceContextBuilder_ == null) {
@@ -1792,7 +1792,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder clearSourceContext() {
       if (sourceContextBuilder_ == null) {
@@ -1810,7 +1810,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder getSourceContextBuilder() {
       
@@ -1822,7 +1822,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder() {
       if (sourceContextBuilder_ != null) {
@@ -1837,7 +1837,7 @@ public  final class Type extends
      * The source context.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
         org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext, org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder> 
@@ -1859,7 +1859,7 @@ public  final class Type extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+     * <code>.google.protobuf.Syntax syntax = 6;</code>
      */
     public int getSyntaxValue() {
       return syntax_;
@@ -1869,7 +1869,7 @@ public  final class Type extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+     * <code>.google.protobuf.Syntax syntax = 6;</code>
      */
     public Builder setSyntaxValue(int value) {
       syntax_ = value;
@@ -1881,7 +1881,7 @@ public  final class Type extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+     * <code>.google.protobuf.Syntax syntax = 6;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -1892,7 +1892,7 @@ public  final class Type extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+     * <code>.google.protobuf.Syntax syntax = 6;</code>
      */
     public Builder setSyntax(org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax value) {
       if (value == null) {
@@ -1908,7 +1908,7 @@ public  final class Type extends
      * The source syntax.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+     * <code>.google.protobuf.Syntax syntax = 6;</code>
      */
     public Builder clearSyntax() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeOrBuilder.java
index aa83b6f..fe845dd 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeOrBuilder.java
@@ -12,7 +12,7 @@ public interface TypeOrBuilder extends
    * The fully qualified message name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   java.lang.String getName();
   /**
@@ -20,7 +20,7 @@ public interface TypeOrBuilder extends
    * The fully qualified message name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
@@ -153,7 +153,7 @@ public interface TypeOrBuilder extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   boolean hasSourceContext();
   /**
@@ -161,7 +161,7 @@ public interface TypeOrBuilder extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext();
   /**
@@ -169,7 +169,7 @@ public interface TypeOrBuilder extends
    * The source context.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder();
 
@@ -178,7 +178,7 @@ public interface TypeOrBuilder extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+   * <code>.google.protobuf.Syntax syntax = 6;</code>
    */
   int getSyntaxValue();
   /**
@@ -186,7 +186,7 @@ public interface TypeOrBuilder extends
    * The source syntax.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 6;</code>
+   * <code>.google.protobuf.Syntax syntax = 6;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeProto.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeProto.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeProto.java
index d5733df..e7c3094 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeProto.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/TypeProto.java
@@ -84,9 +84,10 @@ public final class TypeProto {
       "\003 \003(\0132\027.google.protobuf.Option\";\n\006Option" +
       "\022\014\n\004name\030\001 \001(\t\022#\n\005value\030\002 \001(\0132\024.google.p" +
       "rotobuf.Any*.\n\006Syntax\022\021\n\rSYNTAX_PROTO2\020\000" +
-      "\022\021\n\rSYNTAX_PROTO3\020\001BL\n\023com.google.protob" +
-      "ufB\tTypeProtoP\001\370\001\001\242\002\003GPB\252\002\036Google.Protob" +
-      "uf.WellKnownTypesb\006proto3"
+      "\022\021\n\rSYNTAX_PROTO3\020\001B}\n\023com.google.protob" +
+      "ufB\tTypeProtoP\001Z/google.golang.org/genpr" +
+      "oto/protobuf/ptype;ptype\370\001\001\242\002\003GPB\252\002\036Goog" +
+      "le.Protobuf.WellKnownTypesb\006proto3"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32Value.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32Value.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32Value.java
index 716cd7d..7b4dae9 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32Value.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32Value.java
@@ -83,7 +83,7 @@ public  final class UInt32Value extends
    * The uint32 value.
    * </pre>
    *
-   * <code>optional uint32 value = 1;</code>
+   * <code>uint32 value = 1;</code>
    */
   public int getValue() {
     return value_;
@@ -142,7 +142,7 @@ public  final class UInt32Value extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + getValue();
     hash = (29 * hash) + unknownFields.hashCode();
@@ -369,7 +369,7 @@ public  final class UInt32Value extends
      * The uint32 value.
      * </pre>
      *
-     * <code>optional uint32 value = 1;</code>
+     * <code>uint32 value = 1;</code>
      */
     public int getValue() {
       return value_;
@@ -379,7 +379,7 @@ public  final class UInt32Value extends
      * The uint32 value.
      * </pre>
      *
-     * <code>optional uint32 value = 1;</code>
+     * <code>uint32 value = 1;</code>
      */
     public Builder setValue(int value) {
       
@@ -392,7 +392,7 @@ public  final class UInt32Value extends
      * The uint32 value.
      * </pre>
      *
-     * <code>optional uint32 value = 1;</code>
+     * <code>uint32 value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32ValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32ValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32ValueOrBuilder.java
index d52f0e0..ad0d537 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32ValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt32ValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface UInt32ValueOrBuilder extends
    * The uint32 value.
    * </pre>
    *
-   * <code>optional uint32 value = 1;</code>
+   * <code>uint32 value = 1;</code>
    */
   int getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64Value.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64Value.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64Value.java
index 7ed4029..578a6ff 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64Value.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64Value.java
@@ -83,7 +83,7 @@ public  final class UInt64Value extends
    * The uint64 value.
    * </pre>
    *
-   * <code>optional uint64 value = 1;</code>
+   * <code>uint64 value = 1;</code>
    */
   public long getValue() {
     return value_;
@@ -142,7 +142,7 @@ public  final class UInt64Value extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
         getValue());
@@ -370,7 +370,7 @@ public  final class UInt64Value extends
      * The uint64 value.
      * </pre>
      *
-     * <code>optional uint64 value = 1;</code>
+     * <code>uint64 value = 1;</code>
      */
     public long getValue() {
       return value_;
@@ -380,7 +380,7 @@ public  final class UInt64Value extends
      * The uint64 value.
      * </pre>
      *
-     * <code>optional uint64 value = 1;</code>
+     * <code>uint64 value = 1;</code>
      */
     public Builder setValue(long value) {
       
@@ -393,7 +393,7 @@ public  final class UInt64Value extends
      * The uint64 value.
      * </pre>
      *
-     * <code>optional uint64 value = 1;</code>
+     * <code>uint64 value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64ValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64ValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64ValueOrBuilder.java
index bbdc59a..5f65674 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64ValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UInt64ValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface UInt64ValueOrBuilder extends
    * The uint64 value.
    * </pre>
    *
-   * <code>optional uint64 value = 1;</code>
+   * <code>uint64 value = 1;</code>
    */
   long getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSet.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSet.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSet.java
index 47b9e12..148a3dd 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSet.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSet.java
@@ -31,7 +31,6 @@
 package org.apache.hadoop.hbase.shaded.com.google.protobuf;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.LimitedInputStream;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -39,6 +38,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -58,7 +58,9 @@ import java.util.TreeMap;
  */
 public final class UnknownFieldSet implements MessageLite {
 
-  private UnknownFieldSet() {}
+  private UnknownFieldSet() {
+    fields = null;
+  }
 
   /** Create a new {@link Builder}. */
   public static Builder newBuilder() {
@@ -82,16 +84,18 @@ public final class UnknownFieldSet implements MessageLite {
     return defaultInstance;
   }
   private static final UnknownFieldSet defaultInstance =
-    new UnknownFieldSet(Collections.<Integer, Field>emptyMap());
+    new UnknownFieldSet(Collections.<Integer, Field>emptyMap(),
+        Collections.<Integer, Field>emptyMap());
 
   /**
    * Construct an {@code UnknownFieldSet} around the given map.  The map is
    * expected to be immutable.
    */
-  private UnknownFieldSet(final Map<Integer, Field> fields) {
+  private UnknownFieldSet(final Map<Integer, Field> fields,
+      final Map<Integer, Field> fieldsDescending) {
     this.fields = fields;
   }
-  private Map<Integer, Field> fields;
+  private final Map<Integer, Field> fields;
 
 
   @Override
@@ -224,10 +228,8 @@ public final class UnknownFieldSet implements MessageLite {
     }
   }
 
-  /**
-   * Get the number of bytes required to encode this set using
-   * {@code MessageSet} wire format.
-   */
+
+  /** Get the number of bytes required to encode this set using {@code MessageSet} wire format. */
   public int getSerializedSizeAsMessageSet() {
     int result = 0;
     for (final Map.Entry<Integer, Field> entry : fields.entrySet()) {
@@ -343,12 +345,13 @@ public final class UnknownFieldSet implements MessageLite {
      */
     @Override
     public UnknownFieldSet build() {
-      getFieldBuilder(0);  // Force lastField to be built.
+      getFieldBuilder(0); // Force lastField to be built.
       final UnknownFieldSet result;
       if (fields.isEmpty()) {
         result = getDefaultInstance();
       } else {
-        result = new UnknownFieldSet(Collections.unmodifiableMap(fields));
+        Map<Integer, Field> descendingFields = null;
+        result = new UnknownFieldSet(Collections.unmodifiableMap(fields), descendingFields);
       }
       fields = null;
       return result;
@@ -363,8 +366,9 @@ public final class UnknownFieldSet implements MessageLite {
     @Override
     public Builder clone() {
       getFieldBuilder(0);  // Force lastField to be built.
+      Map<Integer, Field> descendingFields = null;
       return UnknownFieldSet.newBuilder().mergeFrom(
-          new UnknownFieldSet(fields));
+          new UnknownFieldSet(fields, descendingFields));
     }
 
     @Override
@@ -841,9 +845,10 @@ public final class UnknownFieldSet implements MessageLite {
       }
     }
 
+
     /**
-     * Get the number of bytes required to encode this field, including field
-     * number, using {@code MessageSet} wire format.
+     * Get the number of bytes required to encode this field, including field number, using {@code
+     * MessageSet} wire format.
      */
     public int getSerializedSizeAsMessageSetExtension(final int fieldNumber) {
       int result = 0;
@@ -1022,7 +1027,7 @@ public final class UnknownFieldSet implements MessageLite {
       } catch (InvalidProtocolBufferException e) {
         throw e.setUnfinishedMessage(builder.buildPartial());
       } catch (IOException e) {
-        throw new InvalidProtocolBufferException(e.getMessage())
+        throw new InvalidProtocolBufferException(e)
             .setUnfinishedMessage(builder.buildPartial());
       }
       return builder.buildPartial();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSetLite.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSetLite.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSetLite.java
index e622519..faa4c30 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSetLite.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/UnknownFieldSetLite.java
@@ -176,6 +176,42 @@ public final class UnknownFieldSetLite {
   }
 
   /**
+   * Serializes the set and writes it to {@code output} using {@code MessageSet} wire format.
+   *
+   * <p>For use by generated code only.
+   */
+  public void writeAsMessageSetTo(CodedOutputStream output) throws IOException {
+    for (int i = 0; i < count; i++) {
+      int fieldNumber = WireFormat.getTagFieldNumber(tags[i]);
+      output.writeRawMessageSetExtension(fieldNumber, (ByteString) objects[i]);
+    }
+  }
+
+
+  /**
+   * Get the number of bytes required to encode this field, including field number, using {@code
+   * MessageSet} wire format.
+   */
+  public int getSerializedSizeAsMessageSet() {
+    int size = memoizedSerializedSize;
+    if (size != -1) {
+      return size;
+    }
+    
+    size = 0;
+    for (int i = 0; i < count; i++) {
+      int tag = tags[i];
+      int fieldNumber = WireFormat.getTagFieldNumber(tag);
+      size += CodedOutputStream.computeRawMessageSetExtensionSize(
+          fieldNumber, (ByteString) objects[i]);
+    }
+    
+    memoizedSerializedSize = size;
+    
+    return size;
+  }
+
+  /**
    * Get the number of bytes required to encode this set.
    *
    * <p>For use by generated code only.
@@ -216,6 +252,24 @@ public final class UnknownFieldSetLite {
     
     return size;
   }
+  
+  private static boolean equals(int[] tags1, int[] tags2, int count) {
+    for (int i = 0; i < count; ++i) {
+      if (tags1[i] != tags2[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static boolean equals(Object[] objects1, Object[] objects2, int count) {
+    for (int i = 0; i < count; ++i) {
+      if (!objects1[i].equals(objects2[i])) {
+        return false;
+      }
+    }
+    return true;
+  }
 
   @Override
   public boolean equals(Object obj) {
@@ -233,9 +287,8 @@ public final class UnknownFieldSetLite {
     
     UnknownFieldSetLite other = (UnknownFieldSetLite) obj;    
     if (count != other.count
-        // TODO(dweis): Only have to compare up to count but at worst 2x worse than we need to do.
-        || !Arrays.equals(tags, other.tags)
-        || !Arrays.deepEquals(objects, other.objects)) {
+        || !equals(tags, other.tags, count)
+        || !equals(objects, other.objects, count)) {
       return false;
     }
 
@@ -268,7 +321,8 @@ public final class UnknownFieldSetLite {
     }
   }
 
-  private void storeField(int tag, Object value) {
+  // Package private for unsafe experimental runtime.
+  void storeField(int tag, Object value) {
     ensureCapacity();
     
     tags[count] = tag;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Value.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Value.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Value.java
index 72fd69f..593043b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Value.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Value.java
@@ -175,7 +175,7 @@ public  final class Value extends
    * Represents a null value.
    * </pre>
    *
-   * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+   * <code>.google.protobuf.NullValue null_value = 1;</code>
    */
   public int getNullValueValue() {
     if (kindCase_ == 1) {
@@ -188,7 +188,7 @@ public  final class Value extends
    * Represents a null value.
    * </pre>
    *
-   * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+   * <code>.google.protobuf.NullValue null_value = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.NullValue getNullValue() {
     if (kindCase_ == 1) {
@@ -205,7 +205,7 @@ public  final class Value extends
    * Represents a double value.
    * </pre>
    *
-   * <code>optional double number_value = 2;</code>
+   * <code>double number_value = 2;</code>
    */
   public double getNumberValue() {
     if (kindCase_ == 2) {
@@ -220,7 +220,7 @@ public  final class Value extends
    * Represents a string value.
    * </pre>
    *
-   * <code>optional string string_value = 3;</code>
+   * <code>string string_value = 3;</code>
    */
   public java.lang.String getStringValue() {
     java.lang.Object ref = "";
@@ -244,7 +244,7 @@ public  final class Value extends
    * Represents a string value.
    * </pre>
    *
-   * <code>optional string string_value = 3;</code>
+   * <code>string string_value = 3;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getStringValueBytes() {
@@ -271,7 +271,7 @@ public  final class Value extends
    * Represents a boolean value.
    * </pre>
    *
-   * <code>optional bool bool_value = 4;</code>
+   * <code>bool bool_value = 4;</code>
    */
   public boolean getBoolValue() {
     if (kindCase_ == 4) {
@@ -286,7 +286,7 @@ public  final class Value extends
    * Represents a structured value.
    * </pre>
    *
-   * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+   * <code>.google.protobuf.Struct struct_value = 5;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct getStructValue() {
     if (kindCase_ == 5) {
@@ -299,7 +299,7 @@ public  final class Value extends
    * Represents a structured value.
    * </pre>
    *
-   * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+   * <code>.google.protobuf.Struct struct_value = 5;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.StructOrBuilder getStructValueOrBuilder() {
     if (kindCase_ == 5) {
@@ -314,7 +314,7 @@ public  final class Value extends
    * Represents a repeated `Value`.
    * </pre>
    *
-   * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+   * <code>.google.protobuf.ListValue list_value = 6;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue getListValue() {
     if (kindCase_ == 6) {
@@ -327,7 +327,7 @@ public  final class Value extends
    * Represents a repeated `Value`.
    * </pre>
    *
-   * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+   * <code>.google.protobuf.ListValue list_value = 6;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValueOrBuilder getListValueOrBuilder() {
     if (kindCase_ == 6) {
@@ -458,7 +458,7 @@ public  final class Value extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     switch (kindCase_) {
       case 1:
         hash = (37 * hash) + NULL_VALUE_FIELD_NUMBER;
@@ -784,7 +784,7 @@ public  final class Value extends
      * Represents a null value.
      * </pre>
      *
-     * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+     * <code>.google.protobuf.NullValue null_value = 1;</code>
      */
     public int getNullValueValue() {
       if (kindCase_ == 1) {
@@ -797,7 +797,7 @@ public  final class Value extends
      * Represents a null value.
      * </pre>
      *
-     * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+     * <code>.google.protobuf.NullValue null_value = 1;</code>
      */
     public Builder setNullValueValue(int value) {
       kindCase_ = 1;
@@ -810,7 +810,7 @@ public  final class Value extends
      * Represents a null value.
      * </pre>
      *
-     * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+     * <code>.google.protobuf.NullValue null_value = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.NullValue getNullValue() {
       if (kindCase_ == 1) {
@@ -825,7 +825,7 @@ public  final class Value extends
      * Represents a null value.
      * </pre>
      *
-     * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+     * <code>.google.protobuf.NullValue null_value = 1;</code>
      */
     public Builder setNullValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.NullValue value) {
       if (value == null) {
@@ -841,7 +841,7 @@ public  final class Value extends
      * Represents a null value.
      * </pre>
      *
-     * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+     * <code>.google.protobuf.NullValue null_value = 1;</code>
      */
     public Builder clearNullValue() {
       if (kindCase_ == 1) {
@@ -857,7 +857,7 @@ public  final class Value extends
      * Represents a double value.
      * </pre>
      *
-     * <code>optional double number_value = 2;</code>
+     * <code>double number_value = 2;</code>
      */
     public double getNumberValue() {
       if (kindCase_ == 2) {
@@ -870,7 +870,7 @@ public  final class Value extends
      * Represents a double value.
      * </pre>
      *
-     * <code>optional double number_value = 2;</code>
+     * <code>double number_value = 2;</code>
      */
     public Builder setNumberValue(double value) {
       kindCase_ = 2;
@@ -883,7 +883,7 @@ public  final class Value extends
      * Represents a double value.
      * </pre>
      *
-     * <code>optional double number_value = 2;</code>
+     * <code>double number_value = 2;</code>
      */
     public Builder clearNumberValue() {
       if (kindCase_ == 2) {
@@ -899,7 +899,7 @@ public  final class Value extends
      * Represents a string value.
      * </pre>
      *
-     * <code>optional string string_value = 3;</code>
+     * <code>string string_value = 3;</code>
      */
     public java.lang.String getStringValue() {
       java.lang.Object ref = "";
@@ -923,7 +923,7 @@ public  final class Value extends
      * Represents a string value.
      * </pre>
      *
-     * <code>optional string string_value = 3;</code>
+     * <code>string string_value = 3;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getStringValueBytes() {
@@ -948,7 +948,7 @@ public  final class Value extends
      * Represents a string value.
      * </pre>
      *
-     * <code>optional string string_value = 3;</code>
+     * <code>string string_value = 3;</code>
      */
     public Builder setStringValue(
         java.lang.String value) {
@@ -965,7 +965,7 @@ public  final class Value extends
      * Represents a string value.
      * </pre>
      *
-     * <code>optional string string_value = 3;</code>
+     * <code>string string_value = 3;</code>
      */
     public Builder clearStringValue() {
       if (kindCase_ == 3) {
@@ -980,7 +980,7 @@ public  final class Value extends
      * Represents a string value.
      * </pre>
      *
-     * <code>optional string string_value = 3;</code>
+     * <code>string string_value = 3;</code>
      */
     public Builder setStringValueBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -999,7 +999,7 @@ public  final class Value extends
      * Represents a boolean value.
      * </pre>
      *
-     * <code>optional bool bool_value = 4;</code>
+     * <code>bool bool_value = 4;</code>
      */
     public boolean getBoolValue() {
       if (kindCase_ == 4) {
@@ -1012,7 +1012,7 @@ public  final class Value extends
      * Represents a boolean value.
      * </pre>
      *
-     * <code>optional bool bool_value = 4;</code>
+     * <code>bool bool_value = 4;</code>
      */
     public Builder setBoolValue(boolean value) {
       kindCase_ = 4;
@@ -1025,7 +1025,7 @@ public  final class Value extends
      * Represents a boolean value.
      * </pre>
      *
-     * <code>optional bool bool_value = 4;</code>
+     * <code>bool bool_value = 4;</code>
      */
     public Builder clearBoolValue() {
       if (kindCase_ == 4) {
@@ -1043,7 +1043,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct getStructValue() {
       if (structValueBuilder_ == null) {
@@ -1063,7 +1063,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     public Builder setStructValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct value) {
       if (structValueBuilder_ == null) {
@@ -1083,7 +1083,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     public Builder setStructValue(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct.Builder builderForValue) {
@@ -1101,7 +1101,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     public Builder mergeStructValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct value) {
       if (structValueBuilder_ == null) {
@@ -1127,7 +1127,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     public Builder clearStructValue() {
       if (structValueBuilder_ == null) {
@@ -1150,7 +1150,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct.Builder getStructValueBuilder() {
       return getStructValueFieldBuilder().getBuilder();
@@ -1160,7 +1160,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.StructOrBuilder getStructValueOrBuilder() {
       if ((kindCase_ == 5) && (structValueBuilder_ != null)) {
@@ -1177,7 +1177,7 @@ public  final class Value extends
      * Represents a structured value.
      * </pre>
      *
-     * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+     * <code>.google.protobuf.Struct struct_value = 5;</code>
      */
     private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct, org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.StructOrBuilder> 
@@ -1205,7 +1205,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue getListValue() {
       if (listValueBuilder_ == null) {
@@ -1225,7 +1225,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     public Builder setListValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue value) {
       if (listValueBuilder_ == null) {
@@ -1245,7 +1245,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     public Builder setListValue(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue.Builder builderForValue) {
@@ -1263,7 +1263,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     public Builder mergeListValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue value) {
       if (listValueBuilder_ == null) {
@@ -1289,7 +1289,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     public Builder clearListValue() {
       if (listValueBuilder_ == null) {
@@ -1312,7 +1312,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue.Builder getListValueBuilder() {
       return getListValueFieldBuilder().getBuilder();
@@ -1322,7 +1322,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValueOrBuilder getListValueOrBuilder() {
       if ((kindCase_ == 6) && (listValueBuilder_ != null)) {
@@ -1339,7 +1339,7 @@ public  final class Value extends
      * Represents a repeated `Value`.
      * </pre>
      *
-     * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+     * <code>.google.protobuf.ListValue list_value = 6;</code>
      */
     private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue, org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValueOrBuilder> 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ValueOrBuilder.java
index 630afb0..32d390f 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface ValueOrBuilder extends
    * Represents a null value.
    * </pre>
    *
-   * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+   * <code>.google.protobuf.NullValue null_value = 1;</code>
    */
   int getNullValueValue();
   /**
@@ -20,7 +20,7 @@ public interface ValueOrBuilder extends
    * Represents a null value.
    * </pre>
    *
-   * <code>optional .google.protobuf.NullValue null_value = 1;</code>
+   * <code>.google.protobuf.NullValue null_value = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.NullValue getNullValue();
 
@@ -29,7 +29,7 @@ public interface ValueOrBuilder extends
    * Represents a double value.
    * </pre>
    *
-   * <code>optional double number_value = 2;</code>
+   * <code>double number_value = 2;</code>
    */
   double getNumberValue();
 
@@ -38,7 +38,7 @@ public interface ValueOrBuilder extends
    * Represents a string value.
    * </pre>
    *
-   * <code>optional string string_value = 3;</code>
+   * <code>string string_value = 3;</code>
    */
   java.lang.String getStringValue();
   /**
@@ -46,7 +46,7 @@ public interface ValueOrBuilder extends
    * Represents a string value.
    * </pre>
    *
-   * <code>optional string string_value = 3;</code>
+   * <code>string string_value = 3;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getStringValueBytes();
@@ -56,7 +56,7 @@ public interface ValueOrBuilder extends
    * Represents a boolean value.
    * </pre>
    *
-   * <code>optional bool bool_value = 4;</code>
+   * <code>bool bool_value = 4;</code>
    */
   boolean getBoolValue();
 
@@ -65,7 +65,7 @@ public interface ValueOrBuilder extends
    * Represents a structured value.
    * </pre>
    *
-   * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+   * <code>.google.protobuf.Struct struct_value = 5;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Struct getStructValue();
   /**
@@ -73,7 +73,7 @@ public interface ValueOrBuilder extends
    * Represents a structured value.
    * </pre>
    *
-   * <code>optional .google.protobuf.Struct struct_value = 5;</code>
+   * <code>.google.protobuf.Struct struct_value = 5;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.StructOrBuilder getStructValueOrBuilder();
 
@@ -82,7 +82,7 @@ public interface ValueOrBuilder extends
    * Represents a repeated `Value`.
    * </pre>
    *
-   * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+   * <code>.google.protobuf.ListValue list_value = 6;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValue getListValue();
   /**
@@ -90,7 +90,7 @@ public interface ValueOrBuilder extends
    * Represents a repeated `Value`.
    * </pre>
    *
-   * <code>optional .google.protobuf.ListValue list_value = 6;</code>
+   * <code>.google.protobuf.ListValue list_value = 6;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ListValueOrBuilder getListValueOrBuilder();
 


[34/50] [abbrv] hbase git commit: HBASE-17895 TestAsyncProcess#testAction fails if unsafe support is false

Posted by sy...@apache.org.
HBASE-17895 TestAsyncProcess#testAction fails if unsafe support is false

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/hbase-12439
Commit: 23249eb0f5466b3608d80847b398b38b698fcf95
Parents: 3aadc67
Author: AShiou <aa...@gmail.com>
Authored: Tue Apr 11 23:03:48 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Wed Apr 12 11:42:13 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/client/TestAsyncProcess.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/23249eb0/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 3139af1..6c5c1e4 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -1185,8 +1185,8 @@ public class TestAsyncProcess {
     assertTrue(action_2.equals(action_3));
     assertFalse(action_0.equals(action_3));
     assertEquals(0, action_0.compareTo(action_0));
-    assertEquals(-1, action_0.compareTo(action_1));
-    assertEquals(1, action_1.compareTo(action_0));
+    assertTrue(action_0.compareTo(action_1) < 0);
+    assertTrue(action_1.compareTo(action_0) > 0);
     assertEquals(0, action_1.compareTo(action_2));
   }
 


[03/50] [abbrv] hbase git commit: HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
index 47ab440..a5f81e6 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
@@ -305,7 +305,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getLabelCount() > 0) {
         hash = (37 * hash) + LABEL_FIELD_NUMBER;
         hash = (53 * hash) + getLabelList().hashCode();
@@ -884,7 +884,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasExpression()) {
         hash = (37 * hash) + EXPRESSION_FIELD_NUMBER;
         hash = (53 * hash) + getExpression().hashCode();
@@ -1474,7 +1474,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFamily()) {
         hash = (37 * hash) + FAMILY_FIELD_NUMBER;
         hash = (53 * hash) + getFamily().hashCode();
@@ -2776,7 +2776,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRow()) {
         hash = (37 * hash) + ROW_FIELD_NUMBER;
         hash = (53 * hash) + getRow().hashCode();
@@ -5132,7 +5132,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getCellCount() > 0) {
         hash = (37 * hash) + CELL_FIELD_NUMBER;
         hash = (53 * hash) + getCellList().hashCode();
@@ -6313,7 +6313,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -7048,7 +7048,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasResult()) {
         hash = (37 * hash) + RESULT_FIELD_NUMBER;
         hash = (53 * hash) + getResult().hashCode();
@@ -7831,7 +7831,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRow()) {
         hash = (37 * hash) + ROW_FIELD_NUMBER;
         hash = (53 * hash) + getRow().hashCode();
@@ -9553,7 +9553,7 @@ public final class ClientProtos {
             return memoizedHashCode;
           }
           int hash = 41;
-          hash = (19 * hash) + getDescriptorForType().hashCode();
+          hash = (19 * hash) + getDescriptor().hashCode();
           if (hasQualifier()) {
             hash = (37 * hash) + QUALIFIER_FIELD_NUMBER;
             hash = (53 * hash) + getQualifier().hashCode();
@@ -10176,7 +10176,7 @@ public final class ClientProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasFamily()) {
           hash = (37 * hash) + FAMILY_FIELD_NUMBER;
           hash = (53 * hash) + getFamily().hashCode();
@@ -11150,7 +11150,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRow()) {
         hash = (37 * hash) + ROW_FIELD_NUMBER;
         hash = (53 * hash) + getRow().hashCode();
@@ -12875,7 +12875,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -13864,7 +13864,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasResult()) {
         hash = (37 * hash) + RESULT_FIELD_NUMBER;
         hash = (53 * hash) + getResult().hashCode();
@@ -15723,7 +15723,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getColumnCount() > 0) {
         hash = (37 * hash) + COLUMN_FIELD_NUMBER;
         hash = (53 * hash) + getColumnList().hashCode();
@@ -18654,7 +18654,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -20639,7 +20639,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getCellsPerResultCount() > 0) {
         hash = (37 * hash) + CELLS_PER_RESULT_FIELD_NUMBER;
         hash = (53 * hash) + getCellsPerResultList().hashCode();
@@ -22663,7 +22663,7 @@ public final class ClientProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasFamily()) {
           hash = (37 * hash) + FAMILY_FIELD_NUMBER;
           hash = (53 * hash) + getFamily().hashCode();
@@ -23348,7 +23348,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -24539,7 +24539,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLoaded()) {
         hash = (37 * hash) + LOADED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -25199,7 +25199,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasIdentifier()) {
         hash = (37 * hash) + IDENTIFIER_FIELD_NUMBER;
         hash = (53 * hash) + getIdentifier().hashCode();
@@ -25988,7 +25988,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -26736,7 +26736,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBulkToken()) {
         hash = (37 * hash) + BULK_TOKEN_FIELD_NUMBER;
         hash = (53 * hash) + getBulkToken().hashCode();
@@ -27352,7 +27352,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBulkToken()) {
         hash = (37 * hash) + BULK_TOKEN_FIELD_NUMBER;
         hash = (53 * hash) + getBulkToken().hashCode();
@@ -27968,7 +27968,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -28591,7 +28591,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRow()) {
         hash = (37 * hash) + ROW_FIELD_NUMBER;
         hash = (53 * hash) + getRow().hashCode();
@@ -29325,7 +29325,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasValue()) {
         hash = (37 * hash) + VALUE_FIELD_NUMBER;
         hash = (53 * hash) + getValue().hashCode();
@@ -29983,7 +29983,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -30788,7 +30788,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -31720,7 +31720,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasIndex()) {
         hash = (37 * hash) + INDEX_FIELD_NUMBER;
         hash = (53 * hash) + getIndex();
@@ -32820,7 +32820,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -33885,7 +33885,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasMemstoreLoad()) {
         hash = (37 * hash) + MEMSTORELOAD_FIELD_NUMBER;
         hash = (53 * hash) + getMemstoreLoad();
@@ -34622,7 +34622,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionCount() > 0) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegionList().hashCode();
@@ -35941,7 +35941,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasIndex()) {
         hash = (37 * hash) + INDEX_FIELD_NUMBER;
         hash = (53 * hash) + getIndex();
@@ -37215,7 +37215,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getResultOrExceptionCount() > 0) {
         hash = (37 * hash) + RESULTOREXCEPTION_FIELD_NUMBER;
         hash = (53 * hash) + getResultOrExceptionList().hashCode();
@@ -38274,7 +38274,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionActionCount() > 0) {
         hash = (37 * hash) + REGIONACTION_FIELD_NUMBER;
         hash = (53 * hash) + getRegionActionList().hashCode();
@@ -39354,7 +39354,7 @@ public final class ClientProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionActionResultCount() > 0) {
         hash = (37 * hash) + REGIONACTIONRESULT_FIELD_NUMBER;
         hash = (53 * hash) + getRegionActionResultList().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java
index 326f217..83f3281 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterIdProtos.java
@@ -240,7 +240,7 @@ public final class ClusterIdProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasClusterId()) {
         hash = (37 * hash) + CLUSTER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getClusterId().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java
index 97c6d05..4f67ecf 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClusterStatusProtos.java
@@ -622,7 +622,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegionInfo()) {
         hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getRegionInfo().hashCode();
@@ -1379,7 +1379,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSpec()) {
         hash = (37 * hash) + SPEC_FIELD_NUMBER;
         hash = (53 * hash) + getSpec().hashCode();
@@ -2147,7 +2147,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFamilyName()) {
         hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getFamilyName().hashCode();
@@ -2770,7 +2770,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLastFlushedSequenceId()) {
         hash = (37 * hash) + LAST_FLUSHED_SEQUENCE_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -4701,7 +4701,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegionSpecifier()) {
         hash = (37 * hash) + REGION_SPECIFIER_FIELD_NUMBER;
         hash = (53 * hash) + getRegionSpecifier().hashCode();
@@ -6762,7 +6762,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasAgeOfLastAppliedOp()) {
         hash = (37 * hash) + AGEOFLASTAPPLIEDOP_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -7501,7 +7501,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerID()) {
         hash = (37 * hash) + PEERID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerID().hashCode();
@@ -9119,7 +9119,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNumberOfRequests()) {
         hash = (37 * hash) + NUMBER_OF_REQUESTS_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -11442,7 +11442,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServer()) {
         hash = (37 * hash) + SERVER_FIELD_NUMBER;
         hash = (53 * hash) + getServer().hashCode();
@@ -12785,7 +12785,7 @@ public final class ClusterStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasHbaseVersion()) {
         hash = (37 * hash) + HBASE_VERSION_FIELD_NUMBER;
         hash = (53 * hash) + getHbaseVersion().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java
index 1c6c373..dd3faed 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ComparatorProtos.java
@@ -252,7 +252,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -815,7 +815,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasValue()) {
         hash = (37 * hash) + VALUE_FIELD_NUMBER;
         hash = (53 * hash) + getValue().hashCode();
@@ -1305,7 +1305,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasComparable()) {
         hash = (37 * hash) + COMPARABLE_FIELD_NUMBER;
         hash = (53 * hash) + getComparable().hashCode();
@@ -1890,7 +1890,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasComparable()) {
         hash = (37 * hash) + COMPARABLE_FIELD_NUMBER;
         hash = (53 * hash) + getComparable().hashCode();
@@ -2475,7 +2475,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasComparable()) {
         hash = (37 * hash) + COMPARABLE_FIELD_NUMBER;
         hash = (53 * hash) + getComparable().hashCode();
@@ -3211,7 +3211,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasComparable()) {
         hash = (37 * hash) + COMPARABLE_FIELD_NUMBER;
         hash = (53 * hash) + getComparable().hashCode();
@@ -3783,7 +3783,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4434,7 +4434,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPattern()) {
         hash = (37 * hash) + PATTERN_FIELD_NUMBER;
         hash = (53 * hash) + getPattern().hashCode();
@@ -5218,7 +5218,7 @@ public final class ComparatorProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSubstr()) {
         hash = (37 * hash) + SUBSTR_FIELD_NUMBER;
         hash = (53 * hash) + getSubstr().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java
index b195b5a..eb48984 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/EncryptionProtos.java
@@ -386,7 +386,7 @@ public final class EncryptionProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasAlgorithm()) {
         hash = (37 * hash) + ALGORITHM_FIELD_NUMBER;
         hash = (53 * hash) + getAlgorithm().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java
index ccb7e86..64e439a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ErrorHandlingProtos.java
@@ -402,7 +402,7 @@ public final class ErrorHandlingProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDeclaringClass()) {
         hash = (37 * hash) + DECLARING_CLASS_FIELD_NUMBER;
         hash = (53 * hash) + getDeclaringClass().hashCode();
@@ -1384,7 +1384,7 @@ public final class ErrorHandlingProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasClassName()) {
         hash = (37 * hash) + CLASS_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getClassName().hashCode();
@@ -2426,7 +2426,7 @@ public final class ErrorHandlingProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSource()) {
         hash = (37 * hash) + SOURCE_FIELD_NUMBER;
         hash = (53 * hash) + getSource().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java
index cbe92ca..d34c110 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FSProtos.java
@@ -215,7 +215,7 @@ public final class FSProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasVersion()) {
         hash = (37 * hash) + VERSION_FIELD_NUMBER;
         hash = (53 * hash) + getVersion().hashCode();
@@ -886,7 +886,7 @@ public final class FSProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSplitkey()) {
         hash = (37 * hash) + SPLITKEY_FIELD_NUMBER;
         hash = (53 * hash) + getSplitkey().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java
index b329e45..a462d04 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/FilterProtos.java
@@ -252,7 +252,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -819,7 +819,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLimit()) {
         hash = (37 * hash) + LIMIT_FIELD_NUMBER;
         hash = (53 * hash) + getLimit();
@@ -1376,7 +1376,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLimit()) {
         hash = (37 * hash) + LIMIT_FIELD_NUMBER;
         hash = (53 * hash) + getLimit();
@@ -1942,7 +1942,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPrefix()) {
         hash = (37 * hash) + PREFIX_FIELD_NUMBER;
         hash = (53 * hash) + getPrefix().hashCode();
@@ -2540,7 +2540,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasMinColumn()) {
         hash = (37 * hash) + MIN_COLUMN_FIELD_NUMBER;
         hash = (53 * hash) + getMinColumn().hashCode();
@@ -3224,7 +3224,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCompareOp()) {
         hash = (37 * hash) + COMPARE_OP_FIELD_NUMBER;
         hash = (53 * hash) + compareOp_;
@@ -3993,7 +3993,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCompareFilter()) {
         hash = (37 * hash) + COMPARE_FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getCompareFilter().hashCode();
@@ -4727,7 +4727,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCompareFilter()) {
         hash = (37 * hash) + COMPARE_FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getCompareFilter().hashCode();
@@ -5481,7 +5481,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasOperator()) {
         hash = (37 * hash) + OPERATOR_FIELD_NUMBER;
         hash = (53 * hash) + operator_;
@@ -6270,7 +6270,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFilter()) {
         hash = (37 * hash) + FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getFilter().hashCode();
@@ -6793,7 +6793,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -7228,7 +7228,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getQualifiersCount() > 0) {
         hash = (37 * hash) + QUALIFIERS_FIELD_NUMBER;
         hash = (53 * hash) + getQualifiersList().hashCode();
@@ -7784,7 +7784,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getFuzzyKeysDataCount() > 0) {
         hash = (37 * hash) + FUZZY_KEYS_DATA_FIELD_NUMBER;
         hash = (53 * hash) + getFuzzyKeysDataList().hashCode();
@@ -8494,7 +8494,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasStopRowKey()) {
         hash = (37 * hash) + STOP_ROW_KEY_FIELD_NUMBER;
         hash = (53 * hash) + getStopRowKey().hashCode();
@@ -8967,7 +8967,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLenAsVal()) {
         hash = (37 * hash) + LEN_AS_VAL_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -9455,7 +9455,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getSortedPrefixesCount() > 0) {
         hash = (37 * hash) + SORTED_PREFIXES_FIELD_NUMBER;
         hash = (53 * hash) + getSortedPrefixesList().hashCode();
@@ -9971,7 +9971,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPageSize()) {
         hash = (37 * hash) + PAGE_SIZE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -10441,7 +10441,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPrefix()) {
         hash = (37 * hash) + PREFIX_FIELD_NUMBER;
         hash = (53 * hash) + getPrefix().hashCode();
@@ -10935,7 +10935,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCompareFilter()) {
         hash = (37 * hash) + COMPARE_FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getCompareFilter().hashCode();
@@ -11508,7 +11508,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasChance()) {
         hash = (37 * hash) + CHANCE_FIELD_NUMBER;
         hash = (53 * hash) + java.lang.Float.floatToIntBits(
@@ -12003,7 +12003,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCompareFilter()) {
         hash = (37 * hash) + COMPARE_FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getCompareFilter().hashCode();
@@ -12595,7 +12595,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSingleColumnValueFilter()) {
         hash = (37 * hash) + SINGLE_COLUMN_VALUE_FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getSingleColumnValueFilter().hashCode();
@@ -13407,7 +13407,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasColumnFamily()) {
         hash = (37 * hash) + COLUMN_FAMILY_FIELD_NUMBER;
         hash = (53 * hash) + getColumnFamily().hashCode();
@@ -14239,7 +14239,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFilter()) {
         hash = (37 * hash) + FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getFilter().hashCode();
@@ -14891,7 +14891,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getTimestampsCount() > 0) {
         hash = (37 * hash) + TIMESTAMPS_FIELD_NUMBER;
         hash = (53 * hash) + getTimestampsList().hashCode();
@@ -15470,7 +15470,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCompareFilter()) {
         hash = (37 * hash) + COMPARE_FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getCompareFilter().hashCode();
@@ -16062,7 +16062,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFilter()) {
         hash = (37 * hash) + FILTER_FIELD_NUMBER;
         hash = (53 * hash) + getFilter().hashCode();
@@ -16585,7 +16585,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -17128,7 +17128,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasStartRow()) {
         hash = (37 * hash) + START_ROW_FIELD_NUMBER;
         hash = (53 * hash) + getStartRow().hashCode();
@@ -17775,7 +17775,7 @@ public final class FilterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRowRangeListCount() > 0) {
         hash = (37 * hash) + ROW_RANGE_LIST_FIELD_NUMBER;
         hash = (53 * hash) + getRowRangeListList().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java
index 0617426..5ed1187 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java
@@ -503,7 +503,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespace()) {
         hash = (37 * hash) + NAMESPACE_FIELD_NUMBER;
         hash = (53 * hash) + getNamespace().hashCode();
@@ -1321,7 +1321,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -2908,7 +2908,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasState()) {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + state_;
@@ -3585,7 +3585,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -4931,7 +4931,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegionId()) {
         hash = (37 * hash) + REGION_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -5834,7 +5834,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getFavoredNodeCount() > 0) {
         hash = (37 * hash) + FAVORED_NODE_FIELD_NUMBER;
         hash = (53 * hash) + getFavoredNodeList().hashCode();
@@ -6719,7 +6719,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasType()) {
         hash = (37 * hash) + TYPE_FIELD_NUMBER;
         hash = (53 * hash) + type_;
@@ -7301,7 +7301,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFrom()) {
         hash = (37 * hash) + FROM_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -7893,7 +7893,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasColumnFamily()) {
         hash = (37 * hash) + COLUMN_FAMILY_FIELD_NUMBER;
         hash = (53 * hash) + getColumnFamily().hashCode();
@@ -8637,7 +8637,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasHostName()) {
         hash = (37 * hash) + HOST_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getHostName().hashCode();
@@ -9284,7 +9284,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -9913,7 +9913,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -10600,7 +10600,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -11213,7 +11213,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFirst()) {
         hash = (37 * hash) + FIRST_FIELD_NUMBER;
         hash = (53 * hash) + getFirst().hashCode();
@@ -11810,7 +11810,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -12812,7 +12812,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -13970,7 +13970,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSignature()) {
         hash = (37 * hash) + SIGNATURE_FIELD_NUMBER;
         hash = (53 * hash) + getSignature().hashCode();
@@ -14922,7 +14922,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -15343,7 +15343,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLongMsg()) {
         hash = (37 * hash) + LONG_MSG_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -15819,7 +15819,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDoubleMsg()) {
         hash = (37 * hash) + DOUBLE_MSG_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -16293,7 +16293,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBigdecimalMsg()) {
         hash = (37 * hash) + BIGDECIMAL_MSG_FIELD_NUMBER;
         hash = (53 * hash) + getBigdecimalMsg().hashCode();
@@ -16815,7 +16815,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLeastSigBits()) {
         hash = (37 * hash) + LEAST_SIG_BITS_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -17425,7 +17425,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasName()) {
         hash = (37 * hash) + NAME_FIELD_NUMBER;
         hash = (53 * hash) + getName().hashCode();
@@ -18702,7 +18702,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasVersion()) {
         hash = (37 * hash) + VERSION_FIELD_NUMBER;
         hash = (53 * hash) + getVersion().hashCode();
@@ -19851,7 +19851,7 @@ public final class HBaseProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasInfoPort()) {
         hash = (37 * hash) + INFOPORT_FIELD_NUMBER;
         hash = (53 * hash) + getInfoPort();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java
index 00e7080..1de62a2 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HFileProtos.java
@@ -222,7 +222,7 @@ public final class HFileProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getMapEntryCount() > 0) {
         hash = (37 * hash) + MAP_ENTRY_FIELD_NUMBER;
         hash = (53 * hash) + getMapEntryList().hashCode();
@@ -1476,7 +1476,7 @@ public final class HFileProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFileInfoOffset()) {
         hash = (37 * hash) + FILE_INFO_OFFSET_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java
index 0b941d8..fd83795 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LoadBalancerProtos.java
@@ -174,7 +174,7 @@ public final class LoadBalancerProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBalancerOn()) {
         hash = (37 * hash) + BALANCER_ON_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LockServiceProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LockServiceProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LockServiceProtos.java
index 577b680..6dbf9b2 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LockServiceProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/LockServiceProtos.java
@@ -658,7 +658,7 @@ public final class LockServiceProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLockType()) {
         hash = (37 * hash) + LOCK_TYPE_FIELD_NUMBER;
         hash = (53 * hash) + lockType_;
@@ -1845,7 +1845,7 @@ public final class LockServiceProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -2361,7 +2361,7 @@ public final class LockServiceProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -3035,7 +3035,7 @@ public final class LockServiceProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLockStatus()) {
         hash = (37 * hash) + LOCK_STATUS_FIELD_NUMBER;
         hash = (53 * hash) + lockStatus_;
@@ -3921,7 +3921,7 @@ public final class LockServiceProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLockType()) {
         hash = (37 * hash) + LOCK_TYPE_FIELD_NUMBER;
         hash = (53 * hash) + lockType_;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java
index 09605a2..cc06b3b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MapReduceProtos.java
@@ -212,7 +212,7 @@ public final class MapReduceProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getMetricsCount() > 0) {
         hash = (37 * hash) + METRICS_FIELD_NUMBER;
         hash = (53 * hash) + getMetricsList().hashCode();
@@ -1079,7 +1079,7 @@ public final class MapReduceProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getLocationsCount() > 0) {
         hash = (37 * hash) + LOCATIONS_FIELD_NUMBER;
         hash = (53 * hash) + getLocationsList().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
index f4f6a8c..3f8a65b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
@@ -2548,7 +2548,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -3755,7 +3755,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -4950,7 +4950,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -6326,7 +6326,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -7355,7 +7355,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceDescriptor()) {
         hash = (37 * hash) + NAMESPACE_DESCRIPTOR_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceDescriptor().hashCode();
@@ -8012,7 +8012,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceDescriptor()) {
         hash = (37 * hash) + NAMESPACE_DESCRIPTOR_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceDescriptor().hashCode();
@@ -8825,7 +8825,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceName()) {
         hash = (37 * hash) + NAMESPACE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceName().hashCode();
@@ -9709,7 +9709,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -10937,7 +10937,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -12144,7 +12144,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -13191,7 +13191,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -14091,7 +14091,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -15045,7 +15045,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasParentRegionName()) {
         hash = (37 * hash) + PARENT_REGION_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getParentRegionName().hashCode();
@@ -16081,7 +16081,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -18029,7 +18029,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -20273,7 +20273,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -21502,7 +21502,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -22831,7 +22831,7 @@ public final class MasterProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServerName()) {
         hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getServerName().hashCode();


[41/50] [abbrv] hbase git commit: HBASE-17904 Get runs into NoSuchElementException when using Read Replica, with hbase. ipc.client.specificThreadForWriting to be true and hbase.rpc.client.impl to be org.apache.hadoop.hbase.ipc.RpcClientImpl (Huaxiang Sun

Posted by sy...@apache.org.
HBASE-17904 Get runs into NoSuchElementException when using Read Replica, with hbase. ipc.client.specificThreadForWriting
to be true and hbase.rpc.client.impl to be org.apache.hadoop.hbase.ipc.RpcClientImpl (Huaxiang Sun)


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

Branch: refs/heads/hbase-12439
Commit: 7678855fac011a9c02e5d6a42470c0178482a4ce
Parents: 0cd4cec
Author: Michael Stack <st...@apache.org>
Authored: Sun Apr 16 11:00:57 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Sun Apr 16 11:01:06 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/BlockingRpcConnection.java |  2 +-
 .../hbase/client/TestReplicaWithCluster.java    | 50 ++++++++++++++++++++
 2 files changed, 51 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7678855f/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
index 15eb10c..1012ad0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
@@ -156,7 +156,7 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
     }
 
     public void remove(Call call) {
-      callsToWrite.remove();
+      callsToWrite.remove(call);
       // By removing the call from the expected call list, we make the list smaller, but
       // it means as well that we don't know how many calls we cancelled.
       calls.remove(call.id);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7678855f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index becb2eb..2c77541 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Waiter;
+
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -515,7 +516,56 @@ public class TestReplicaWithCluster {
 
       Assert.assertTrue(r.isStale());
     } finally {
+      HTU.getAdmin().disableTable(hdt.getTableName());
+      HTU.deleteTable(hdt.getTableName());
+    }
+  }
+
+  @Test
+  public void testReplicaGetWithRpcClientImpl() throws IOException {
+    HTU.getConfiguration().setBoolean("hbase.ipc.client.specificThreadForWriting", true);
+    HTU.getConfiguration().set("hbase.rpc.client.impl", "org.apache.hadoop.hbase.ipc.RpcClientImpl");
+    // Create table then get the single region for our new table.
+    HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaGetWithRpcClientImpl");
+    hdt.setRegionReplication(NB_SERVERS);
+    hdt.addCoprocessor(SlowMeCopro.class.getName());
+
+    try {
+      Table table = HTU.createTable(hdt, new byte[][] { f }, null);
+
+      Put p = new Put(row);
+      p.addColumn(f, row, row);
+      table.put(p);
 
+      // Flush so it can be picked by the replica refresher thread
+      HTU.flush(table.getName());
+
+      // Sleep for some time until data is picked up by replicas
+      try {
+        Thread.sleep(2 * REFRESH_PERIOD);
+      } catch (InterruptedException e1) {
+        LOG.error(e1);
+      }
+
+      try {
+        // Create the new connection so new config can kick in
+        Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration());
+        Table t = connection.getTable(hdt.getTableName());
+
+        // But if we ask for stale we will get it
+        SlowMeCopro.cdl.set(new CountDownLatch(1));
+        Get g = new Get(row);
+        g.setConsistency(Consistency.TIMELINE);
+        Result r = t.get(g);
+        Assert.assertTrue(r.isStale());
+        SlowMeCopro.cdl.get().countDown();
+      } finally {
+        SlowMeCopro.cdl.get().countDown();
+        SlowMeCopro.sleepTime.set(0);
+      }
+    } finally {
+      HTU.getConfiguration().unset("hbase.ipc.client.specificThreadForWriting");
+      HTU.getConfiguration().unset("hbase.rpc.client.impl");
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }


[23/50] [abbrv] hbase git commit: HBASE-17816 HRegion#mutateRowWithLocks should update writeRequestCount metric (Weizhan Zeng)

Posted by sy...@apache.org.
HBASE-17816 HRegion#mutateRowWithLocks should update writeRequestCount metric (Weizhan Zeng)


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

Branch: refs/heads/hbase-12439
Commit: 48b2502a5fcd4d3cd954c3abf6703422da7cdc2f
Parents: af604f0
Author: Jerry He <je...@apache.org>
Authored: Thu Apr 6 16:45:45 2017 -0700
Committer: Jerry He <je...@apache.org>
Committed: Thu Apr 6 16:45:45 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |  1 +
 .../hadoop/hbase/regionserver/TestHRegion.java  | 24 ++++++++++++++++++++
 2 files changed, 25 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/48b2502a/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 7f889ce..a87b679 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
@@ -6966,6 +6966,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public void mutateRowsWithLocks(Collection<Mutation> mutations,
       Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
+    writeRequestsCount.add(mutations.size());
     MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
     processRowsWithLocks(proc, -1, nonceGroup, nonce);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48b2502a/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 eac3c77..d56d6ec 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
@@ -6391,4 +6391,28 @@ public class TestHRegion {
       this.region = null;
     }
   }
+
+  @Test
+  public void testMutateRow_WriteRequestCount() throws Exception {
+    byte[] row1 = Bytes.toBytes("row1");
+    byte[] fam1 = Bytes.toBytes("fam1");
+    byte[] qf1 = Bytes.toBytes("qualifier");
+    byte[] val1 = Bytes.toBytes("value1");
+
+    RowMutations rm = new RowMutations(row1);
+    Put put = new Put(row1);
+    put.addColumn(fam1, qf1, val1);
+    rm.add(put);
+
+    this.region = initHRegion(tableName, method, CONF, fam1);
+    try {
+      long wrcBeforeMutate = this.region.writeRequestsCount.longValue();
+      this.region.mutateRow(rm);
+      long wrcAfterMutate = this.region.writeRequestsCount.longValue();
+      Assert.assertEquals(wrcBeforeMutate + rm.getMutations().size(), wrcAfterMutate);
+    } finally {
+      HBaseTestingUtility.closeRegionAndWAL(this.region);
+      this.region = null;
+    }
+  }
 }


[24/50] [abbrv] hbase git commit: HBASE-17836 CellUtil#estimatedSerializedSizeOf is slow when input is ByteBufferCell

Posted by sy...@apache.org.
HBASE-17836 CellUtil#estimatedSerializedSizeOf is slow when input is ByteBufferCell


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

Branch: refs/heads/hbase-12439
Commit: 1a701ce44484f45a8a07ea9826b84f0df6f1518e
Parents: 48b2502
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Sat Apr 1 13:50:01 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Apr 7 09:30:15 2017 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/CellUtil.java      | 8 +++-----
 .../src/main/java/org/apache/hadoop/hbase/SplitLogTask.java  | 2 +-
 2 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1a701ce4/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 6585173..e1bc969 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -1385,12 +1385,10 @@ public final class CellUtil {
    * @return Estimate of the <code>cell</code> size in bytes.
    */
   public static int estimatedSerializedSizeOf(final Cell cell) {
-    // If a KeyValue, we can give a good estimate of size.
-    if (cell instanceof KeyValue) {
-      return ((KeyValue)cell).getLength() + Bytes.SIZEOF_INT;
+    if (cell instanceof ExtendedCell) {
+      return ((ExtendedCell) cell).getSerializedSize(true) + Bytes.SIZEOF_INT;
     }
-    // TODO: Should we add to Cell a sizeOf?  Would it help? Does it make sense if Cell is
-    // prefix encoded or compressed?
+
     return getSumOfCellElementLengths(cell) +
       // Use the KeyValue's infrastructure size presuming that another implementation would have
       // same basic cost.

http://git-wip-us.apache.org/repos/asf/hbase/blob/1a701ce4/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
index 03d5108..3ecaa86 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
@@ -86,7 +86,7 @@ public class SplitLogTask {
   public ServerName getServerName() {
     return this.originServer;
   }
-  
+
   public ZooKeeperProtos.SplitLogTask.RecoveryMode getMode() {
     return this.mode;
   }


[50/50] [abbrv] hbase git commit: Revert "HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)"

Posted by sy...@apache.org.
Revert "HBASE-16438 Create a cell type so that chunk id is embedded in it (Ram)"

This reverts commit c2c2178b2eebe4439eadec6b37fae2566944c16b.


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

Branch: refs/heads/hbase-12439
Commit: ecdfb82326035ad8221940919bbeb3fe16ec2658
Parents: c2c2178
Author: Ramkrishna <ra...@intel.com>
Authored: Tue Apr 18 00:00:12 2017 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Tue Apr 18 00:00:12 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  24 ++
 .../org/apache/hadoop/hbase/ExtendedCell.java   |  10 -
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 -
 .../hbase/regionserver/ByteBufferChunkCell.java |  48 ---
 .../apache/hadoop/hbase/regionserver/Chunk.java |  60 +--
 .../hadoop/hbase/regionserver/ChunkCreator.java | 404 -------------------
 .../hbase/regionserver/HRegionServer.java       |  14 +-
 .../hbase/regionserver/MemStoreChunkPool.java   | 265 ++++++++++++
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |   4 +-
 .../hbase/regionserver/MemStoreLABImpl.java     | 171 ++++----
 .../regionserver/NoTagByteBufferChunkCell.java  |  48 ---
 .../hadoop/hbase/regionserver/OffheapChunk.java |  31 +-
 .../hadoop/hbase/regionserver/OnheapChunk.java  |  32 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   3 -
 .../coprocessor/TestCoprocessorInterface.java   |   4 -
 .../TestRegionObserverScannerOpenHook.java      |   3 -
 .../coprocessor/TestRegionObserverStacking.java |   3 -
 .../io/hfile/TestScannerFromBucketCache.java    |   3 -
 .../hadoop/hbase/master/TestCatalogJanitor.java |   7 -
 .../hadoop/hbase/regionserver/TestBulkLoad.java |   2 +-
 .../hbase/regionserver/TestCellFlatSet.java     |   2 +-
 .../regionserver/TestCompactingMemStore.java    |  37 +-
 .../TestCompactingToCellArrayMapMemStore.java   |  16 +-
 .../TestCompactionArchiveConcurrentClose.java   |   1 -
 .../TestCompactionArchiveIOException.java       |   1 -
 .../regionserver/TestCompactionPolicy.java      |   1 -
 .../hbase/regionserver/TestDefaultMemStore.java |  14 +-
 .../regionserver/TestFailedAppendAndSync.java   |   1 -
 .../hbase/regionserver/TestHMobStore.java       |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   2 -
 .../regionserver/TestHRegionReplayEvents.java   |   2 +-
 .../regionserver/TestMemStoreChunkPool.java     |  48 +--
 .../hbase/regionserver/TestMemStoreLAB.java     |  27 +-
 .../TestMemstoreLABWithoutPool.java             | 168 --------
 .../hbase/regionserver/TestRecoveredEdits.java  |   1 -
 .../hbase/regionserver/TestRegionIncrement.java |   1 -
 .../hadoop/hbase/regionserver/TestStore.java    |   1 -
 .../TestStoreFileRefresherChore.java            |   1 -
 .../hbase/regionserver/TestWALLockup.java       |   1 -
 .../TestWALMonotonicallyIncreasingSeqId.java    |   1 -
 .../hbase/regionserver/wal/TestDurability.java  |   3 -
 41 files changed, 479 insertions(+), 990 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 56de21b..e1bc969 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -3135,4 +3135,28 @@ public final class CellUtil {
       return Type.DeleteFamily.getCode();
     }
   }
+
+  /**
+   * Clone the passed cell by copying its data into the passed buf.
+   */
+  public static Cell copyCellTo(Cell cell, ByteBuffer buf, int offset, int len) {
+    int tagsLen = cell.getTagsLength();
+    if (cell instanceof ExtendedCell) {
+      ((ExtendedCell) cell).write(buf, offset);
+    } else {
+      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
+      // other case also. The data fragments within Cell is copied into buf as in KeyValue
+      // serialization format only.
+      KeyValueUtil.appendTo(cell, buf, offset, true);
+    }
+    if (tagsLen == 0) {
+      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
+      // which directly return tagsLen as 0. So we avoid parsing many length components in
+      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
+      // call getTagsLength().
+      return new NoTagsByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
+    } else {
+      return new ByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
index 10f20ca..517873f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.io.HeapSize;
 public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestamp, HeapSize,
     Cloneable {
 
-  public static int CELL_NOT_BASED_ON_CHUNK = -1;
   /**
    * Write this cell to an OutputStream in a {@link KeyValue} format.
    * <br> KeyValue format <br>
@@ -74,13 +73,4 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
    * @return The deep cloned cell
    */
   Cell deepClone();
-
-  /**
-   * Extracts the id of the backing bytebuffer of this cell if it was obtained from fixed sized
-   * chunks as in case of MemstoreLAB
-   * @return the chunk id if the cell is backed by fixed sized Chunks, else return -1
-   */
-  default int getChunkId() {
-    return CELL_NOT_BASED_ON_CHUNK;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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 f9670e1..bb9f282 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
@@ -748,8 +748,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
-    // Initialize the chunkCreator
-    initializeMemStoreChunkCreator();
     this.fileSystemManager = new MasterFileSystem(this);
     this.walManager = new MasterWalManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
deleted file mode 100644
index a8f1000..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset
- * @see MemStoreLAB
- */
-//TODO : When moving this cell to CellChunkMap we will have the following things
-// to be serialized
-// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
-@InterfaceAudience.Private
-public class ByteBufferChunkCell extends ByteBufferKeyValue {
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
index fc4aa0b..2cbf0a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
@@ -21,10 +21,8 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
 /**
  * A chunk of memory out of which allocations are sliced.
@@ -48,41 +46,13 @@ public abstract class Chunk {
   /** Size of chunk in bytes */
   protected final int size;
 
-  // The unique id associated with the chunk.
-  private final int id;
-
-  // indicates if the chunk is formed by ChunkCreator#MemstorePool
-  private final boolean fromPool;
-
-  /**
-   * Create an uninitialized chunk. Note that memory is not allocated yet, so
-   * this is cheap.
-   * @param size in bytes
-   * @param id the chunk id
-   */
-  public Chunk(int size, int id) {
-    this(size, id, false);
-  }
-
   /**
-   * Create an uninitialized chunk. Note that memory is not allocated yet, so
-   * this is cheap.
+   * Create an uninitialized chunk. Note that memory is not allocated yet, so this is cheap.
+   *
    * @param size in bytes
-   * @param id the chunk id
-   * @param fromPool if the chunk is formed by pool
    */
-  public Chunk(int size, int id, boolean fromPool) {
+  Chunk(int size) {
     this.size = size;
-    this.id = id;
-    this.fromPool = fromPool;
-  }
-
-  int getId() {
-    return this.id;
-  }
-
-  boolean isFromPool() {
-    return this.fromPool;
   }
 
   /**
@@ -90,24 +60,7 @@ public abstract class Chunk {
    * constructed the chunk. It is thread-safe against other threads calling alloc(), who will block
    * until the allocation is complete.
    */
-  public void init() {
-    assert nextFreeOffset.get() == UNINITIALIZED;
-    try {
-      allocateDataBuffer();
-    } catch (OutOfMemoryError e) {
-      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
-      assert failInit; // should be true.
-      throw e;
-    }
-    // Mark that it's ready for use
-    // Move 8 bytes since the first 8 bytes are having the chunkid in it
-    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, Bytes.SIZEOF_LONG);
-    // We should always succeed the above CAS since only one thread
-    // calls init()!
-    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
-  }
-
-  abstract void allocateDataBuffer();
+  public abstract void init();
 
   /**
    * Reset the offset to UNINITIALIZED before before reusing an old chunk
@@ -121,8 +74,7 @@ public abstract class Chunk {
 
   /**
    * Try to allocate <code>size</code> bytes from the chunk.
-   * If a chunk is tried to get allocated before init() call, the thread doing the allocation
-   * will be in busy-wait state as it will keep looping till the nextFreeOffset is set.
+   *
    * @return the offset of the successful allocation, or -1 to indicate not-enough-space
    */
   public int alloc(int size) {
@@ -144,7 +96,7 @@ public abstract class Chunk {
       if (oldOffset + size > data.capacity()) {
         return -1; // alloc doesn't fit
       }
-      // TODO : If seqID is to be written add 8 bytes here for nextFreeOFfset
+
       // Try to atomically claim this chunk
       if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size)) {
         // we got the alloc

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
deleted file mode 100644
index 073fb25..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ /dev/null
@@ -1,404 +0,0 @@
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.lang.ref.SoftReference;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Does the management of memstoreLAB chunk creations. A monotonically incrementing id is associated
- * with every chunk
- */
-@InterfaceAudience.Private
-public class ChunkCreator {
-  private final Log LOG = LogFactory.getLog(ChunkCreator.class);
-  // monotonically increasing chunkid
-  private AtomicInteger chunkID = new AtomicInteger(1);
-  // maps the chunk against the monotonically increasing chunk id. We need to preserve the
-  // natural ordering of the key
-  // CellChunkMap creation should convert the soft ref to hard reference
-  private Map<Integer, SoftReference<Chunk>> chunkIdMap =
-      new ConcurrentHashMap<Integer, SoftReference<Chunk>>();
-  private final int chunkSize;
-  private final boolean offheap;
-  @VisibleForTesting
-  static ChunkCreator INSTANCE;
-  @VisibleForTesting
-  static boolean chunkPoolDisabled = false;
-  private MemStoreChunkPool pool;
-
-  @VisibleForTesting
-  ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
-    this.chunkSize = chunkSize;
-    this.offheap = offheap;
-    this.pool = initializePool(globalMemStoreSize, poolSizePercentage, initialCountPercentage);
-    if (heapMemoryManager != null && this.pool != null) {
-      // Register with Heap Memory manager
-      heapMemoryManager.registerTuneObserver(this.pool);
-    }
-  }
-
-  /**
-   * Initializes the instance of MSLABChunkCreator
-   * @param chunkSize the chunkSize
-   * @param offheap indicates if the chunk is to be created offheap or not
-   * @param globalMemStoreSize  the global memstore size
-   * @param poolSizePercentage pool size percentage
-   * @param initialCountPercentage the initial count of the chunk pool if any
-   * @param heapMemoryManager the heapmemory manager
-   * @return singleton MSLABChunkCreator
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
-      justification = "Method is called by single thread at the starting of RS")
-  @VisibleForTesting
-  public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
-      float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager) {
-    if (INSTANCE != null) return INSTANCE;
-    INSTANCE = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
-        initialCountPercentage, heapMemoryManager);
-    return INSTANCE;
-  }
-
-  static ChunkCreator getInstance() {
-    return INSTANCE;
-  }
-
-  /**
-   * Creates and inits a chunk.
-   * @return the chunk that was initialized
-   */
-  Chunk getChunk() {
-    Chunk chunk = null;
-    if (pool != null) {
-      //  the pool creates the chunk internally. The chunk#init() call happens here
-      chunk = this.pool.getChunk();
-      // the pool has run out of maxCount
-      if (chunk == null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("The chunk pool is full. Reached maxCount= " + this.pool.getMaxCount()
-              + ". Creating chunk onheap.");
-        }
-      }
-    }
-    if (chunk == null) {
-      chunk = createChunk();
-    }
-    // put this chunk into the chunkIdMap
-    this.chunkIdMap.put(chunk.getId(), new SoftReference<>(chunk));
-    // now we need to actually do the expensive memory allocation step in case of a new chunk,
-    // else only the offset is set to the beginning of the chunk to accept allocations
-    chunk.init();
-    return chunk;
-  }
-
-  private Chunk createChunk() {
-    return createChunk(false);
-  }
-
-  /**
-   * Creates the chunk either onheap or offheap
-   * @param pool indicates if the chunks have to be created which will be used by the Pool
-   * @return the chunk
-   */
-  private Chunk createChunk(boolean pool) {
-    int id = chunkID.getAndIncrement();
-    assert id > 0;
-    // do not create offheap chunk on demand
-    if (pool && this.offheap) {
-      return new OffheapChunk(chunkSize, id, pool);
-    } else {
-      return new OnheapChunk(chunkSize, id, pool);
-    }
-  }
-
-  @VisibleForTesting
-  // TODO : To be used by CellChunkMap
-  Chunk getChunk(int id) {
-    SoftReference<Chunk> ref = chunkIdMap.get(id);
-    if (ref != null) {
-      return ref.get();
-    }
-    return null;
-  }
-
-  int getChunkSize() {
-    return this.chunkSize;
-  }
-
-  boolean isOffheap() {
-    return this.offheap;
-  }
-
-  private void removeChunks(Set<Integer> chunkIDs) {
-    this.chunkIdMap.keySet().removeAll(chunkIDs);
-  }
-
-  Chunk removeChunk(int chunkId) {
-    SoftReference<Chunk> ref = this.chunkIdMap.remove(chunkId);
-    if (ref != null) {
-      return ref.get();
-    }
-    return null;
-  }
-
-  @VisibleForTesting
-  int size() {
-    return this.chunkIdMap.size();
-  }
-
-  @VisibleForTesting
-  void clearChunkIds() {
-    this.chunkIdMap.clear();
-  }
-
-  /**
-   * A pool of {@link Chunk} instances.
-   *
-   * MemStoreChunkPool caches a number of retired chunks for reusing, it could
-   * decrease allocating bytes when writing, thereby optimizing the garbage
-   * collection on JVM.
-   */
-  private  class MemStoreChunkPool implements HeapMemoryTuneObserver {
-    private int maxCount;
-
-    // A queue of reclaimed chunks
-    private final BlockingQueue<Chunk> reclaimedChunks;
-    private final float poolSizePercentage;
-
-    /** Statistics thread schedule pool */
-    private final ScheduledExecutorService scheduleThreadPool;
-    /** Statistics thread */
-    private static final int statThreadPeriod = 60 * 5;
-    private final AtomicLong chunkCount = new AtomicLong();
-    private final AtomicLong reusedChunkCount = new AtomicLong();
-
-    MemStoreChunkPool(int maxCount, int initialCount, float poolSizePercentage) {
-      this.maxCount = maxCount;
-      this.poolSizePercentage = poolSizePercentage;
-      this.reclaimedChunks = new LinkedBlockingQueue<>();
-      for (int i = 0; i < initialCount; i++) {
-        Chunk chunk = createChunk(true);
-        chunk.init();
-        reclaimedChunks.add(chunk);
-      }
-      chunkCount.set(initialCount);
-      final String n = Thread.currentThread().getName();
-      scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
-          .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
-      this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
-          statThreadPeriod, TimeUnit.SECONDS);
-    }
-
-    /**
-     * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
-     * not yet created max allowed chunks count. When we have already created max allowed chunks and
-     * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
-     * then.
-     * Note: Chunks returned by this pool must be put back to the pool after its use.
-     * @return a chunk
-     * @see #putbackChunks(Set)
-     */
-    Chunk getChunk() {
-      Chunk chunk = reclaimedChunks.poll();
-      if (chunk != null) {
-        chunk.reset();
-        reusedChunkCount.incrementAndGet();
-      } else {
-        // Make a chunk iff we have not yet created the maxCount chunks
-        while (true) {
-          long created = this.chunkCount.get();
-          if (created < this.maxCount) {
-            if (this.chunkCount.compareAndSet(created, created + 1)) {
-              chunk = createChunk(true);
-              break;
-            }
-          } else {
-            break;
-          }
-        }
-      }
-      return chunk;
-    }
-
-    /**
-     * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
-     * chunks
-     * @param chunks
-     */
-    private void putbackChunks(Set<Integer> chunks) {
-      int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
-      Iterator<Integer> iterator = chunks.iterator();
-      while (iterator.hasNext()) {
-        Integer chunkId = iterator.next();
-        // remove the chunks every time though they are from the pool or not
-        Chunk chunk = ChunkCreator.this.removeChunk(chunkId);
-        if (chunk != null) {
-          if (chunk.isFromPool() && toAdd > 0) {
-            reclaimedChunks.add(chunk);
-          }
-          toAdd--;
-        }
-      }
-    }
-
-    private class StatisticsThread extends Thread {
-      StatisticsThread() {
-        super("MemStoreChunkPool.StatisticsThread");
-        setDaemon(true);
-      }
-
-      @Override
-      public void run() {
-        logStats();
-      }
-
-      private void logStats() {
-        if (!LOG.isDebugEnabled()) return;
-        long created = chunkCount.get();
-        long reused = reusedChunkCount.get();
-        long total = created + reused;
-        LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
-            + ",created chunk count=" + created
-            + ",reused chunk count=" + reused
-            + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
-                (float) reused / (float) total, 2)));
-      }
-    }
-
-    private int getMaxCount() {
-      return this.maxCount;
-    }
-
-    @Override
-    public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
-      // don't do any tuning in case of offheap memstore
-      if (isOffheap()) {
-        LOG.warn("Not tuning the chunk pool as it is offheap");
-        return;
-      }
-      int newMaxCount =
-          (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
-      if (newMaxCount != this.maxCount) {
-        // We need an adjustment in the chunks numbers
-        if (newMaxCount > this.maxCount) {
-          // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
-          // create and add them to Q
-          LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
-          this.maxCount = newMaxCount;
-        } else {
-          // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
-          // itself. If the extra chunks are serving already, do not pool those when we get them back
-          LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
-          this.maxCount = newMaxCount;
-          if (this.reclaimedChunks.size() > newMaxCount) {
-            synchronized (this) {
-              while (this.reclaimedChunks.size() > newMaxCount) {
-                this.reclaimedChunks.poll();
-              }
-            }
-          }
-        }
-      }
-    }
-  }
-
-  @VisibleForTesting
-  static void clearDisableFlag() {
-    chunkPoolDisabled = false;
-  }
-
-  private MemStoreChunkPool initializePool(long globalMemStoreSize, float poolSizePercentage,
-      float initialCountPercentage) {
-    if (poolSizePercentage <= 0) {
-      LOG.info("PoolSizePercentage is less than 0. So not using pool");
-      return null;
-    }
-    if (chunkPoolDisabled) {
-      return null;
-    }
-    if (poolSizePercentage > 1.0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / getChunkSize());
-    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
-      throw new IllegalArgumentException(
-          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
-    }
-    int initialCount = (int) (initialCountPercentage * maxCount);
-    LOG.info("Allocating MemStoreChunkPool with chunk size "
-        + StringUtils.byteDesc(getChunkSize()) + ", max count " + maxCount
-        + ", initial count " + initialCount);
-    return new MemStoreChunkPool(maxCount, initialCount, poolSizePercentage);
-  }
-
-  @VisibleForTesting
-  int getMaxCount() {
-    if (pool != null) {
-      return pool.getMaxCount();
-    }
-    return 0;
-  }
-
-  @VisibleForTesting
-  int getPoolSize() {
-    if (pool != null) {
-      return pool.reclaimedChunks.size();
-    }
-    return 0;
-  }
-
-  /*
-   * Only used in testing
-   */
-  @VisibleForTesting
-  void clearChunksInPool() {
-    if (pool != null) {
-      pool.reclaimedChunks.clear();
-    }
-  }
-
-  synchronized void putbackChunks(Set<Integer> chunks) {
-    if (pool != null) {
-      pool.putbackChunks(chunks);
-    } else {
-      this.removeChunks(chunks);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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 41eb0a3..b3b5113 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
@@ -1472,7 +1472,7 @@ public class HRegionServer extends HasThread implements
       startServiceThreads();
       startHeapMemoryManager();
       // Call it after starting HeapMemoryManager.
-      initializeMemStoreChunkCreator();
+      initializeMemStoreChunkPool();
       LOG.info("Serving as " + this.serverName +
         ", RpcServer on " + rpcServices.isa +
         ", sessionid=0x" +
@@ -1492,7 +1492,7 @@ public class HRegionServer extends HasThread implements
     }
   }
 
-  protected void initializeMemStoreChunkCreator() {
+  private void initializeMemStoreChunkPool() {
     if (MemStoreLAB.isEnabled(conf)) {
       // MSLAB is enabled. So initialize MemStoreChunkPool
       // By this time, the MemstoreFlusher is already initialized. We can get the global limits from
@@ -1506,10 +1506,12 @@ public class HRegionServer extends HasThread implements
       float initialCountPercentage = conf.getFloat(MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY,
           MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT);
       int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY, MemStoreLAB.CHUNK_SIZE_DEFAULT);
-      // init the chunkCreator
-      ChunkCreator chunkCreator =
-          ChunkCreator.initialize(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
-            initialCountPercentage, this.hMemManager);
+      MemStoreChunkPool pool = MemStoreChunkPool.initialize(globalMemStoreSize, poolSizePercentage,
+          initialCountPercentage, chunkSize, offheap);
+      if (pool != null && this.hMemManager != null) {
+        // Register with Heap Memory manager
+        this.hMemManager.registerTuneObserver(pool);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
new file mode 100644
index 0000000..b7ac212
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreChunkPool.java
@@ -0,0 +1,265 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A pool of {@link Chunk} instances.
+ * 
+ * MemStoreChunkPool caches a number of retired chunks for reusing, it could
+ * decrease allocating bytes when writing, thereby optimizing the garbage
+ * collection on JVM.
+ * 
+ * The pool instance is globally unique and could be obtained through
+ * {@link MemStoreChunkPool#initialize(long, float, float, int, boolean)}
+ * 
+ * {@link MemStoreChunkPool#getChunk()} is called when MemStoreLAB allocating
+ * bytes, and {@link MemStoreChunkPool#putbackChunks(BlockingQueue)} is called
+ * when MemStore clearing snapshot for flush
+ */
+@SuppressWarnings("javadoc")
+@InterfaceAudience.Private
+public class MemStoreChunkPool implements HeapMemoryTuneObserver {
+  private static final Log LOG = LogFactory.getLog(MemStoreChunkPool.class);
+
+  // Static reference to the MemStoreChunkPool
+  static MemStoreChunkPool GLOBAL_INSTANCE;
+  /** Boolean whether we have disabled the memstore chunk pool entirely. */
+  static boolean chunkPoolDisabled = false;
+
+  private int maxCount;
+
+  // A queue of reclaimed chunks
+  private final BlockingQueue<Chunk> reclaimedChunks;
+  private final int chunkSize;
+  private final float poolSizePercentage;
+
+  /** Statistics thread schedule pool */
+  private final ScheduledExecutorService scheduleThreadPool;
+  /** Statistics thread */
+  private static final int statThreadPeriod = 60 * 5;
+  private final AtomicLong chunkCount = new AtomicLong();
+  private final AtomicLong reusedChunkCount = new AtomicLong();
+  private final boolean offheap;
+
+  MemStoreChunkPool(int chunkSize, int maxCount, int initialCount, float poolSizePercentage,
+      boolean offheap) {
+    this.maxCount = maxCount;
+    this.chunkSize = chunkSize;
+    this.poolSizePercentage = poolSizePercentage;
+    this.offheap = offheap;
+    this.reclaimedChunks = new LinkedBlockingQueue<>();
+    for (int i = 0; i < initialCount; i++) {
+      Chunk chunk = this.offheap ? new OffheapChunk(chunkSize) : new OnheapChunk(chunkSize);
+      chunk.init();
+      reclaimedChunks.add(chunk);
+    }
+    chunkCount.set(initialCount);
+    final String n = Thread.currentThread().getName();
+    scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
+        .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
+    this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
+        statThreadPeriod, TimeUnit.SECONDS);
+  }
+
+  /**
+   * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we have
+   * not yet created max allowed chunks count. When we have already created max allowed chunks and
+   * no free chunks as of now, return null. It is the responsibility of the caller to make a chunk
+   * then.
+   * Note: Chunks returned by this pool must be put back to the pool after its use.
+   * @return a chunk
+   * @see #putbackChunk(Chunk)
+   * @see #putbackChunks(BlockingQueue)
+   */
+  Chunk getChunk() {
+    Chunk chunk = reclaimedChunks.poll();
+    if (chunk != null) {
+      chunk.reset();
+      reusedChunkCount.incrementAndGet();
+    } else {
+      // Make a chunk iff we have not yet created the maxCount chunks
+      while (true) {
+        long created = this.chunkCount.get();
+        if (created < this.maxCount) {
+          chunk = this.offheap ? new OffheapChunk(this.chunkSize) : new OnheapChunk(this.chunkSize);
+          if (this.chunkCount.compareAndSet(created, created + 1)) {
+            break;
+          }
+        } else {
+          break;
+        }
+      }
+    }
+    return chunk;
+  }
+
+  /**
+   * Add the chunks to the pool, when the pool achieves the max size, it will
+   * skip the remaining chunks
+   * @param chunks
+   */
+  synchronized void putbackChunks(BlockingQueue<Chunk> chunks) {
+    int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
+    Chunk chunk = null;
+    while ((chunk = chunks.poll()) != null && toAdd > 0) {
+      reclaimedChunks.add(chunk);
+      toAdd--;
+    }
+  }
+
+  /**
+   * Add the chunk to the pool, if the pool has achieved the max size, it will
+   * skip it
+   * @param chunk
+   */
+  synchronized void putbackChunk(Chunk chunk) {
+    if (reclaimedChunks.size() < this.maxCount) {
+      reclaimedChunks.add(chunk);
+    }
+  }
+
+  int getPoolSize() {
+    return this.reclaimedChunks.size();
+  }
+
+  /*
+   * Only used in testing
+   */
+  void clearChunks() {
+    this.reclaimedChunks.clear();
+  }
+
+  private class StatisticsThread extends Thread {
+    StatisticsThread() {
+      super("MemStoreChunkPool.StatisticsThread");
+      setDaemon(true);
+    }
+
+    @Override
+    public void run() {
+      logStats();
+    }
+
+    private void logStats() {
+      if (!LOG.isDebugEnabled()) return;
+      long created = chunkCount.get();
+      long reused = reusedChunkCount.get();
+      long total = created + reused;
+      LOG.debug("Stats: current pool size=" + reclaimedChunks.size()
+          + ",created chunk count=" + created
+          + ",reused chunk count=" + reused
+          + ",reuseRatio=" + (total == 0 ? "0" : StringUtils.formatPercent(
+              (float) reused / (float) total, 2)));
+    }
+  }
+
+  /**
+   * @return the global MemStoreChunkPool instance
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
+      justification = "Method is called by single thread at the starting of RS")
+  static MemStoreChunkPool initialize(long globalMemStoreSize, float poolSizePercentage,
+      float initialCountPercentage, int chunkSize, boolean offheap) {
+    if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
+    if (chunkPoolDisabled) return null;
+
+    if (poolSizePercentage <= 0) {
+      chunkPoolDisabled = true;
+      return null;
+    }
+    if (poolSizePercentage > 1.0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
+    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
+      throw new IllegalArgumentException(
+          MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
+    }
+    int initialCount = (int) (initialCountPercentage * maxCount);
+    LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
+        + ", max count " + maxCount + ", initial count " + initialCount);
+    GLOBAL_INSTANCE = new MemStoreChunkPool(chunkSize, maxCount, initialCount, poolSizePercentage,
+        offheap);
+    return GLOBAL_INSTANCE;
+  }
+
+  /**
+   * @return The singleton instance of this pool.
+   */
+  static MemStoreChunkPool getPool() {
+    return GLOBAL_INSTANCE;
+  }
+
+  int getMaxCount() {
+    return this.maxCount;
+  }
+
+  @VisibleForTesting
+  static void clearDisableFlag() {
+    chunkPoolDisabled = false;
+  }
+
+  @Override
+  public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
+    // don't do any tuning in case of offheap memstore
+    if (this.offheap) {
+      LOG.warn("Not tuning the chunk pool as it is offheap");
+      return;
+    }
+    int newMaxCount = (int) (newMemstoreSize * poolSizePercentage / chunkSize);
+    if (newMaxCount != this.maxCount) {
+      // We need an adjustment in the chunks numbers
+      if (newMaxCount > this.maxCount) {
+        // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
+        // create and add them to Q
+        LOG.info("Max count for chunks increased from " + this.maxCount + " to " + newMaxCount);
+        this.maxCount = newMaxCount;
+      } else {
+        // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
+        // itself. If the extra chunks are serving already, do not pool those when we get them back
+        LOG.info("Max count for chunks decreased from " + this.maxCount + " to " + newMaxCount);
+        this.maxCount = newMaxCount;
+        if (this.reclaimedChunks.size() > newMaxCount) {
+          synchronized (this) {
+            while (this.reclaimedChunks.size() > newMaxCount) {
+              this.reclaimedChunks.poll();
+            }
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
index 72e937c..f6d1607 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * <p>
  * The MemStoreLAB is basically a bump-the-pointer allocator that allocates big (2MB) chunks from
  * and then doles it out to threads that request slices into the array. These chunks can get pooled
- * as well. See {@link ChunkCreator}.
+ * as well. See {@link MemStoreChunkPool}.
  * <p>
  * The purpose of this is to combat heap fragmentation in the regionserver. By ensuring that all
  * Cells in a given memstore refer only to large chunks of contiguous memory, we ensure that
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  * {@link #copyCellInto(Cell)} gets called. This allocates enough size in the chunk to hold this
  * cell's data and copies into this area and then recreate a Cell over this copied data.
  * <p>
- * @see ChunkCreator
+ * @see MemStoreChunkPool
  */
 @InterfaceAudience.Private
 public interface MemStoreLAB {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 4fba82d..4e87135 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -18,26 +18,23 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.nio.ByteBuffer;
-import java.util.Set;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantLock;
 
 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.ExtendedCell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+
 /**
  * A memstore-local allocation buffer.
  * <p>
@@ -58,8 +55,8 @@ import com.google.common.base.Preconditions;
  * would provide a performance improvement - probably would speed up the
  * Bytes.toLong/Bytes.toInt calls in KeyValue, but some of those are cached
  * anyway.
- * The chunks created by this MemStoreLAB can get pooled at {@link ChunkCreator}.
- * When the Chunk comes from pool, it can be either an on heap or an off heap backed chunk. The chunks,
+ * The chunks created by this MemStoreLAB can get pooled at {@link MemStoreChunkPool}.
+ * When the Chunk comes pool, it can be either an on heap or an off heap backed chunk. The chunks,
  * which this MemStoreLAB creates on its own (when no chunk available from pool), those will be
  * always on heap backed.
  */
@@ -69,15 +66,14 @@ public class MemStoreLABImpl implements MemStoreLAB {
   static final Log LOG = LogFactory.getLog(MemStoreLABImpl.class);
 
   private AtomicReference<Chunk> curChunk = new AtomicReference<>();
-  // Lock to manage multiple handlers requesting for a chunk
-  private ReentrantLock lock = new ReentrantLock();
-
-  // A set of chunks contained by this memstore LAB
+  // A queue of chunks from pool contained by this memstore LAB
+  // TODO: in the future, it would be better to have List implementation instead of Queue,
+  // as FIFO order is not so important here
   @VisibleForTesting
-  Set<Integer> chunks = new ConcurrentSkipListSet<Integer>();
+  BlockingQueue<Chunk> pooledChunkQueue = null;
   private final int chunkSize;
   private final int maxAlloc;
-  private final ChunkCreator chunkCreator;
+  private final MemStoreChunkPool chunkPool;
 
   // This flag is for closing this instance, its set when clearing snapshot of
   // memstore
@@ -96,12 +92,20 @@ public class MemStoreLABImpl implements MemStoreLAB {
   public MemStoreLABImpl(Configuration conf) {
     chunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);
     maxAlloc = conf.getInt(MAX_ALLOC_KEY, MAX_ALLOC_DEFAULT);
-    this.chunkCreator = ChunkCreator.getInstance();
+    this.chunkPool = MemStoreChunkPool.getPool();
+    // currently chunkQueue is only used for chunkPool
+    if (this.chunkPool != null) {
+      // set queue length to chunk pool max count to avoid keeping reference of
+      // too many non-reclaimable chunks
+      pooledChunkQueue = new LinkedBlockingQueue<>(chunkPool.getMaxCount());
+    }
+
     // if we don't exclude allocations >CHUNK_SIZE, we'd infiniteloop on one!
     Preconditions.checkArgument(maxAlloc <= chunkSize,
         MAX_ALLOC_KEY + " must be less than " + CHUNK_SIZE_KEY);
   }
 
+
   @Override
   public Cell copyCellInto(Cell cell) {
     int size = KeyValueUtil.length(cell);
@@ -114,52 +118,19 @@ public class MemStoreLABImpl implements MemStoreLAB {
     Chunk c = null;
     int allocOffset = 0;
     while (true) {
-      // Try to get the chunk
       c = getOrMakeChunk();
-      // we may get null because the some other thread succeeded in getting the lock
-      // and so the current thread has to try again to make its chunk or grab the chunk
-      // that the other thread created
       // Try to allocate from this chunk
-      if (c != null) {
-        allocOffset = c.alloc(size);
-        if (allocOffset != -1) {
-          // We succeeded - this is the common case - small alloc
-          // from a big buffer
-          break;
-        }
-        // not enough space!
-        // try to retire this chunk
-        tryRetireChunk(c);
+      allocOffset = c.alloc(size);
+      if (allocOffset != -1) {
+        // We succeeded - this is the common case - small alloc
+        // from a big buffer
+        break;
       }
+      // not enough space!
+      // try to retire this chunk
+      tryRetireChunk(c);
     }
-    return copyToChunkCell(cell, c.getData(), allocOffset, size);
-  }
-
-  /**
-   * Clone the passed cell by copying its data into the passed buf and create a cell with a chunkid
-   * out of it
-   */
-  private Cell copyToChunkCell(Cell cell, ByteBuffer buf, int offset, int len) {
-    int tagsLen = cell.getTagsLength();
-    if (cell instanceof ExtendedCell) {
-      ((ExtendedCell) cell).write(buf, offset);
-    } else {
-      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
-      // other case also. The data fragments within Cell is copied into buf as in KeyValue
-      // serialization format only.
-      KeyValueUtil.appendTo(cell, buf, offset, true);
-    }
-    // TODO : write the seqid here. For writing seqId we should create a new cell type so
-    // that seqId is not used as the state
-    if (tagsLen == 0) {
-      // When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
-      // which directly return tagsLen as 0. So we avoid parsing many length components in
-      // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
-      // call getTagsLength().
-      return new NoTagByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
-    } else {
-      return new ByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
-    }
+    return CellUtil.copyCellTo(cell, c.getData(), allocOffset, size);
   }
 
   /**
@@ -171,9 +142,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
     this.closed = true;
     // We could put back the chunks to pool for reusing only when there is no
     // opening scanner which will read their data
-    int count  = openScannerCount.get();
-    if(count == 0) {
-      recycleChunks();
+    if (chunkPool != null && openScannerCount.get() == 0
+        && reclaimed.compareAndSet(false, true)) {
+      chunkPool.putbackChunks(this.pooledChunkQueue);
     }
   }
 
@@ -191,14 +162,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
   @Override
   public void decScannerCount() {
     int count = this.openScannerCount.decrementAndGet();
-    if (this.closed && count == 0) {
-      recycleChunks();
-    }
-  }
-
-  private void recycleChunks() {
-    if (reclaimed.compareAndSet(false, true)) {
-      chunkCreator.putbackChunks(chunks);
+    if (this.closed && chunkPool != null && count == 0
+        && reclaimed.compareAndSet(false, true)) {
+      chunkPool.putbackChunks(this.pooledChunkQueue);
     }
   }
 
@@ -224,33 +190,45 @@ public class MemStoreLABImpl implements MemStoreLAB {
    * allocate a new one from the JVM.
    */
   private Chunk getOrMakeChunk() {
-    // Try to get the chunk
-    Chunk c = curChunk.get();
-    if (c != null) {
-      return c;
-    }
-    // No current chunk, so we want to allocate one. We race
-    // against other allocators to CAS in an uninitialized chunk
-    // (which is cheap to allocate)
-    if (lock.tryLock()) {
-      try {
-        // once again check inside the lock
-        c = curChunk.get();
-        if (c != null) {
-          return c;
-        }
-        c = this.chunkCreator.getChunk();
-        if (c != null) {
-          // set the curChunk. No need of CAS as only one thread will be here
-          curChunk.set(c);
-          chunks.add(c.getId());
-          return c;
+    while (true) {
+      // Try to get the chunk
+      Chunk c = curChunk.get();
+      if (c != null) {
+        return c;
+      }
+
+      // No current chunk, so we want to allocate one. We race
+      // against other allocators to CAS in an uninitialized chunk
+      // (which is cheap to allocate)
+      if (chunkPool != null) {
+        c = chunkPool.getChunk();
+      }
+      boolean pooledChunk = false;
+      if (c != null) {
+        // This is chunk from pool
+        pooledChunk = true;
+      } else {
+        c = new OnheapChunk(chunkSize);// When chunk is not from pool, always make it as on heap.
+      }
+      if (curChunk.compareAndSet(null, c)) {
+        // we won race - now we need to actually do the expensive
+        // allocation step
+        c.init();
+        if (pooledChunk) {
+          if (!this.closed && !this.pooledChunkQueue.offer(c)) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Chunk queue is full, won't reuse this new chunk. Current queue size: "
+                  + pooledChunkQueue.size());
+            }
+          }
         }
-      } finally {
-        lock.unlock();
+        return c;
+      } else if (pooledChunk) {
+        chunkPool.putbackChunk(c);
       }
+      // someone else won race - that's fine, we'll try to grab theirs
+      // in the next iteration of the loop.
     }
-    return null;
   }
 
   @VisibleForTesting
@@ -258,15 +236,8 @@ public class MemStoreLABImpl implements MemStoreLAB {
     return this.curChunk.get();
   }
 
-  @VisibleForTesting
+
   BlockingQueue<Chunk> getPooledChunks() {
-    BlockingQueue<Chunk> pooledChunks = new LinkedBlockingQueue<>();
-    for (Integer id : this.chunks) {
-      Chunk chunk = chunkCreator.getChunk(id);
-      if (chunk != null && chunk.isFromPool()) {
-        pooledChunks.add(chunk);
-      }
-    }
-    return pooledChunks;
+    return this.pooledChunkQueue;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
deleted file mode 100644
index a8ba50c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
- * @see MemStoreLAB
- */
-@InterfaceAudience.Private
-public class NoTagByteBufferChunkCell extends NoTagsByteBufferKeyValue {
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
index e244a33..ed98cfa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
@@ -21,27 +21,34 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+import com.google.common.base.Preconditions;
+
 /**
  * An off heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OffheapChunk extends Chunk {
 
-  OffheapChunk(int size, int id) {
-    // better if this is always created fromPool. This should not be called
-    super(size, id);
-  }
-
-  OffheapChunk(int size, int id, boolean fromPool) {
-    super(size, id, fromPool);
-    assert fromPool == true;
+  OffheapChunk(int size) {
+    super(size);
   }
 
   @Override
-  void allocateDataBuffer() {
-    if (data == null) {
-      data = ByteBuffer.allocateDirect(this.size);
-      data.putLong(0, this.getId());
+  public void init() {
+    assert nextFreeOffset.get() == UNINITIALIZED;
+    try {
+      if (data == null) {
+        data = ByteBuffer.allocateDirect(this.size);
+      }
+    } catch (OutOfMemoryError e) {
+      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
+      assert failInit; // should be true.
+      throw e;
     }
+    // Mark that it's ready for use
+    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
+    // We should always succeed the above CAS since only one thread
+    // calls init()!
+    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
index da34e24..bd33cb5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
@@ -21,25 +21,33 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+import com.google.common.base.Preconditions;
+
 /**
  * An on heap chunk implementation.
  */
 @InterfaceAudience.Private
 public class OnheapChunk extends Chunk {
 
-  OnheapChunk(int size, int id) {
-    super(size, id);
-  }
-
-  OnheapChunk(int size, int id, boolean fromPool) {
-    super(size, id, fromPool);
+  OnheapChunk(int size) {
+    super(size);
   }
 
-  @Override
-  void allocateDataBuffer() {
-    if (data == null) {
-      data = ByteBuffer.allocate(this.size);
-      data.putLong(0, this.getId());
+  public void init() {
+    assert nextFreeOffset.get() == UNINITIALIZED;
+    try {
+      if (data == null) {
+        data = ByteBuffer.allocate(this.size);
+      }
+    } catch (OutOfMemoryError e) {
+      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
+      assert failInit; // should be true.
+      throw e;
     }
+    // Mark that it's ready for use
+    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
+    // We should always succeed the above CAS since only one thread
+    // calls init()!
+    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 6563122..82c2eab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -96,8 +96,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -2428,7 +2426,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
       final Configuration conf, final HTableDescriptor htd, boolean initialize)
       throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     WAL wal = createWal(conf, rootDir, info);
     return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 8d8b6df..422c54b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -27,7 +27,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -50,10 +49,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -400,7 +397,6 @@ public class TestCoprocessorInterface {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
     Path path = new Path(DIR + callingMethod);
     Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index b99087d..80d0e3a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -47,12 +47,10 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -154,7 +152,6 @@ public class TestRegionObserverScannerOpenHook {
     for (byte[] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     WAL wal = HBaseTestingUtility.createWal(conf, path, info);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
index 15d449d..2e44dee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
@@ -34,9 +34,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -102,7 +100,6 @@ public class TestRegionObserverStacking extends TestCase {
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
index fae7247..f1775d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
@@ -40,10 +40,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -344,7 +342,6 @@ public class TestScannerFromBucketCache {
   private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
       String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
       byte[]... families) throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path logDir = test_util.getDataTestDirOnTestFS(callingMethod + ".log");
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 32bce26..cc73d9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -65,8 +65,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActi
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
 import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -75,7 +73,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Triple;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -91,10 +88,6 @@ public class TestCatalogJanitor {
   @Rule
   public TestName name = new TestName();
 
-  @BeforeClass
-  public static void setup() throws Exception {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-  }
   /**
    * Mock MasterServices for tests below.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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 096c5ef..418aadf 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
@@ -241,7 +241,7 @@ public class TestBulkLoad {
     for (byte[] family : families) {
       hTableDescriptor.addFamily(new HColumnDescriptor(family));
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+
     // TODO We need a way to do this without creating files
     return HRegion.createHRegion(hRegionInfo,
         new Path(testFolder.newFolder().toURI()),

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
index 09877b0..3b4d068 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
@@ -73,7 +73,7 @@ public class TestCellFlatSet extends TestCase {
     descCbOnHeap = new CellArrayMap(CellComparator.COMPARATOR,descCells,0,NUM_OF_CELLS,true);
     CONF.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     CONF.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    ChunkCreator.chunkPoolDisabled = false;
+    MemStoreChunkPool.chunkPoolDisabled = false;
   }
 
   /* Create and test CellSet based on CellArrayMap */

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index 9e90f3e..a888c45 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -51,7 +50,7 @@ import static org.junit.Assert.assertTrue;
 public class TestCompactingMemStore extends TestDefaultMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingMemStore.class);
-  protected static ChunkCreator chunkCreator;
+  protected static MemStoreChunkPool chunkPool;
   protected HRegion region;
   protected RegionServicesForStores regionServicesForStores;
   protected HStore store;
@@ -66,7 +65,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
 
   @After
   public void tearDown() throws Exception {
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
   }
 
   @Override
@@ -85,21 +84,15 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
     HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("foobar"));
-    htd.addFamily(hcd);
-    HRegionInfo info =
-        new HRegionInfo(TableName.valueOf("foobar"), null, null, false);
-    WAL wal = hbaseUtility.createWal(conf, hbaseUtility.getDataTestDir(), info);
-    this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf, htd, wal, true);
-    //this.region = hbaseUtility.createTestRegion("foobar", hcd);
+    this.region = hbaseUtility.createTestRegion("foobar", hcd);
     this.regionServicesForStores = region.getRegionServicesForStores();
     this.store = new HStore(region, hcd, conf);
 
     long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
-      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-    assertTrue(chunkCreator != null);
+    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
+        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
+    assertTrue(chunkPool != null);
   }
 
   /**
@@ -397,7 +390,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -441,16 +434,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
 
@@ -471,7 +464,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   @Test
@@ -523,16 +516,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     memstore.add(new KeyValue(row, fam, qf1, 3, val), null);
     assertEquals(3, memstore.getActive().getCellsCount());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
 
@@ -560,7 +553,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   //////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
index 66e107a..5a48455 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellArrayMapMemStore.java
@@ -44,13 +44,17 @@ import java.util.List;
 public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore {
 
   private static final Log LOG = LogFactory.getLog(TestCompactingToCellArrayMapMemStore.class);
+  //private static MemStoreChunkPool chunkPool;
+  //private HRegion region;
+  //private RegionServicesForStores regionServicesForStores;
+  //private HStore store;
 
   //////////////////////////////////////////////////////////////////////////////
   // Helpers
   //////////////////////////////////////////////////////////////////////////////
 
   @Override public void tearDown() throws Exception {
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
   }
 
   @Override public void setUp() throws Exception {
@@ -404,16 +408,16 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
 
@@ -434,7 +438,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   @Test
@@ -468,7 +472,7 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index e320368..8e85730 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -164,7 +164,6 @@ public class TestCompactionArchiveConcurrentClose {
 
     HRegionFileSystem fs = new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf),
         tableDir, info);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index e7fcf18..89b2368 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -174,7 +174,6 @@ public class TestCompactionArchiveIOException {
   private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
       throws IOException {
     Configuration conf = testUtil.getConfiguration();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
     Path regionDir = new Path(tableDir, info.getEncodedName());
     Path storeDir = new Path(regionDir, htd.getColumnFamilies()[0].getNameAsString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/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
index bff5bec..7154511 100644
--- 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
@@ -104,7 +104,6 @@ public class TestCompactionPolicy {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
 
     hlog = new FSHLog(fs, basedir, logName, conf);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
     region.close();
     Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 41b304b..7434eb1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -85,7 +84,6 @@ public class TestDefaultMemStore {
   protected static final byte[] FAMILY = Bytes.toBytes("column");
   protected MultiVersionConcurrencyControl mvcc;
   protected AtomicLong startSeqNum = new AtomicLong(0);
-  protected ChunkCreator chunkCreator;
 
   private String getName() {
     return this.name.getMethodName();
@@ -94,17 +92,9 @@ public class TestDefaultMemStore {
   @Before
   public void setUp() throws Exception {
     internalSetUp();
-    // no pool
-    this.chunkCreator =
-        ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     this.memstore = new DefaultMemStore();
   }
 
-  @AfterClass
-  public static void tearDownClass() throws Exception {
-    ChunkCreator.getInstance().clearChunkIds();
-  }
-
   protected void internalSetUp() throws Exception {
     this.mvcc = new MultiVersionConcurrencyControl();
   }
@@ -139,9 +129,7 @@ public class TestDefaultMemStore {
       assertEquals(Segment.getCellLength(kv), sizeChangeForSecondCell.getDataSize());
       // make sure chunk size increased even when writing the same cell, if using MSLAB
       if (msLab instanceof MemStoreLABImpl) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        assertEquals(2 * Segment.getCellLength(kv) + Bytes.SIZEOF_LONG,
+        assertEquals(2 * Segment.getCellLength(kv),
           ((MemStoreLABImpl) msLab).getCurrentChunk().getNextFreeOffset());
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 24e850d..73fb9cf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -266,7 +266,6 @@ public class TestFailedAppendAndSync {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
   throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 0f24a24..b416c7d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -153,7 +153,7 @@ public class TestHMobStore {
 
     htd.addFamily(hcd);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
     final WALFactory wals = new WALFactory(walConf, null, methodName);


[36/50] [abbrv] hbase git commit: HBASE-16775 Fix flaky TestExportSnapshot#testExportRetry.

Posted by sy...@apache.org.
HBASE-16775 Fix flaky TestExportSnapshot#testExportRetry.

Reason for flakyness: Current test is probability based fault injection and triggers failure 3% of the time. Earlier when test used LocalJobRunner which didn't honor "mapreduce.map.maxattempts", it'd pass 97% time (when no fault is injected) and fail 3% time (when fault was injected). Point being, even when the test was complete wrong, we couldn't catch it because it was probability based.

This change will inject fault in a deterministic manner.
On design side, it encapsulates all testing hooks in ExportSnapshot.java into single inner class.

Change-Id: Icba866e1d56a5281748df89f4dd374bc45bad249


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

Branch: refs/heads/hbase-12439
Commit: da5fb27eabed4a4b4d251be973ee945fb52895bf
Parents: cf3215d
Author: Apekshit Sharma <ap...@apache.org>
Authored: Thu Oct 6 14:20:58 2016 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Wed Apr 12 11:11:31 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/snapshot/ExportSnapshot.java   | 58 +++++++-------
 .../hbase/snapshot/TestExportSnapshot.java      | 84 +++++++++++---------
 .../snapshot/TestExportSnapshotNoCluster.java   |  2 +-
 .../hbase/snapshot/TestMobExportSnapshot.java   |  7 +-
 .../snapshot/TestMobSecureExportSnapshot.java   |  7 +-
 .../snapshot/TestSecureExportSnapshot.java      |  7 +-
 6 files changed, 93 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index e2086e9..e3ad951 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -29,7 +29,6 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -110,9 +109,12 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
   private static final String CONF_BANDWIDTH_MB = "snapshot.export.map.bandwidth.mb";
   protected static final String CONF_SKIP_TMP = "snapshot.export.skip.tmp";
 
-  static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
-  static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry";
-
+  static class Testing {
+    static final String CONF_TEST_FAILURE = "test.snapshot.export.failure";
+    static final String CONF_TEST_FAILURE_COUNT = "test.snapshot.export.failure.count";
+    int failuresCountToInject = 0;
+    int injectedFailureCount = 0;
+  }
 
   // Command line options and defaults.
   static final class Options {
@@ -149,12 +151,10 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
 
   private static class ExportMapper extends Mapper<BytesWritable, NullWritable,
                                                    NullWritable, NullWritable> {
+    private static final Log LOG = LogFactory.getLog(ExportMapper.class);
     final static int REPORT_SIZE = 1 * 1024 * 1024;
     final static int BUFFER_SIZE = 64 * 1024;
 
-    private boolean testFailures;
-    private Random random;
-
     private boolean verifyChecksum;
     private String filesGroup;
     private String filesUser;
@@ -169,9 +169,12 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
     private Path inputArchive;
     private Path inputRoot;
 
+    private static Testing testing = new Testing();
+
     @Override
     public void setup(Context context) throws IOException {
       Configuration conf = context.getConfiguration();
+
       Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX);
       Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX);
 
@@ -186,8 +189,6 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
       outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY);
 
-      testFailures = conf.getBoolean(CONF_TEST_FAILURE, false);
-
       try {
         srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
         inputFs = FileSystem.get(inputRoot.toUri(), srcConf);
@@ -210,6 +211,12 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       for (Counter c : Counter.values()) {
         context.getCounter(c).increment(0);
       }
+      if (context.getConfiguration().getBoolean(Testing.CONF_TEST_FAILURE, false)) {
+        testing.failuresCountToInject = conf.getInt(Testing.CONF_TEST_FAILURE_COUNT, 0);
+        // Get number of times we have already injected failure based on attempt number of this
+        // task.
+        testing.injectedFailureCount = context.getTaskAttemptID().getId();
+      }
     }
 
     @Override
@@ -251,35 +258,23 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       return new Path(outputArchive, path);
     }
 
-    /*
-     * Used by TestExportSnapshot to simulate a failure
+    /**
+     * Used by TestExportSnapshot to test for retries when failures happen.
+     * Failure is injected in {@link #copyFile(Context, SnapshotFileInfo, Path)}.
      */
     private void injectTestFailure(final Context context, final SnapshotFileInfo inputInfo)
         throws IOException {
-      if (testFailures) {
-        if (context.getConfiguration().getBoolean(CONF_TEST_RETRY, false)) {
-          if (random == null) {
-            random = new Random();
-          }
-
-          // FLAKY-TEST-WARN: lower is better, we can get some runs without the
-          // retry, but at least we reduce the number of test failures due to
-          // this test exception from the same map task.
-          if (random.nextFloat() < 0.03) {
-            throw new IOException("TEST RETRY FAILURE: Unable to copy input=" + inputInfo
-                                  + " time=" + System.currentTimeMillis());
-          }
-        } else {
-          context.getCounter(Counter.COPY_FAILED).increment(1);
-          throw new IOException("TEST FAILURE: Unable to copy input=" + inputInfo);
-        }
-      }
+      if (!context.getConfiguration().getBoolean(Testing.CONF_TEST_FAILURE, false)) return;
+      if (testing.injectedFailureCount >= testing.failuresCountToInject) return;
+      testing.injectedFailureCount++;
+      context.getCounter(Counter.COPY_FAILED).increment(1);
+      LOG.debug("Injecting failure. Count: " + testing.injectedFailureCount);
+      throw new IOException(String.format("TEST FAILURE (%d of max %d): Unable to copy input=%s",
+          testing.injectedFailureCount, testing.failuresCountToInject, inputInfo));
     }
 
     private void copyFile(final Context context, final SnapshotFileInfo inputInfo,
         final Path outputPath) throws IOException {
-      injectTestFailure(context, inputInfo);
-
       // Get the file information
       FileStatus inputStat = getSourceFileStatus(context, inputInfo);
 
@@ -318,6 +313,7 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
         }
       } finally {
         in.close();
+        injectTestFailure(context, inputInfo);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
index 1beb518..52412d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.snapshot;
 import static org.apache.hadoop.util.ToolRunner.run;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 import java.io.IOException;
 import java.net.URI;
@@ -44,12 +45,10 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -57,6 +56,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;
 
 /**
@@ -72,6 +72,9 @@ public class TestExportSnapshot {
 
   protected final static byte[] FAMILY = Bytes.toBytes("cf");
 
+  @Rule
+  public final TestName testName = new TestName();
+
   protected TableName tableName;
   private byte[] emptySnapshotName;
   private byte[] snapshotName;
@@ -85,17 +88,27 @@ public class TestExportSnapshot {
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
     conf.setBoolean("hbase.master.enabletable.roundrobin", true);
     conf.setInt("mapreduce.map.maxattempts", 10);
+    // If a single node has enough failures (default 3), resource manager will blacklist it.
+    // With only 2 nodes and tests injecting faults, we don't want that.
+    conf.setInt("mapreduce.job.maxtaskfailures.per.tracker", 100);
   }
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
-    TEST_UTIL.startMiniCluster(3);
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniHBaseCluster();
+    TEST_UTIL.shutdownMiniMapReduceCluster();
+    TEST_UTIL.shutdownMiniZKCluster();
   }
 
   /**
@@ -105,10 +118,9 @@ public class TestExportSnapshot {
   public void setUp() throws Exception {
     this.admin = TEST_UTIL.getAdmin();
 
-    long tid = System.currentTimeMillis();
-    tableName = TableName.valueOf("testtb-" + tid);
-    snapshotName = Bytes.toBytes("snaptb0-" + tid);
-    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + tid);
+    tableName = TableName.valueOf("testtb-" + testName.getMethodName());
+    snapshotName = Bytes.toBytes("snaptb0-" + testName.getMethodName());
+    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + testName.getMethodName());
 
     // create Table
     createTable();
@@ -191,16 +203,16 @@ public class TestExportSnapshot {
       Path copyDir, boolean overwrite) throws Exception {
     testExportFileSystemState(TEST_UTIL.getConfiguration(), tableName, snapshotName, targetName,
       filesExpected, TEST_UTIL.getDefaultRootDirPath(), copyDir,
-      overwrite, getBypassRegionPredicate());
+      overwrite, getBypassRegionPredicate(), true);
   }
 
   /**
-   * Test ExportSnapshot
+   * Creates destination directory, runs ExportSnapshot() tool, and runs some verifications.
    */
   protected static void testExportFileSystemState(final Configuration conf, final TableName tableName,
       final byte[] snapshotName, final byte[] targetName, final int filesExpected,
       final Path sourceDir, Path copyDir, final boolean overwrite,
-      final RegionPredicate bypassregionPredicate) throws Exception {
+      final RegionPredicate bypassregionPredicate, boolean success) throws Exception {
     URI hdfsUri = FileSystem.get(conf).getUri();
     FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
     copyDir = copyDir.makeQualified(fs);
@@ -218,7 +230,12 @@ public class TestExportSnapshot {
 
     // Export Snapshot
     int res = run(conf, new ExportSnapshot(), opts.toArray(new String[opts.size()]));
-    assertEquals(0, res);
+    assertEquals(success ? 0 : 1, res);
+    if (!success) {
+      final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
+      assertFalse(fs.exists(new Path(copyDir, targetDir)));
+      return;
+    }
 
     // Verify File-System state
     FileStatus[] rootFiles = fs.listStatus(copyDir);
@@ -242,42 +259,35 @@ public class TestExportSnapshot {
   }
 
   /**
-   * Check that ExportSnapshot will return a failure if something fails.
+   * Check that ExportSnapshot will succeed if something fails but the retry succeed.
    */
   @Test
-  public void testExportFailure() throws Exception {
-    assertEquals(1, runExportAndInjectFailures(snapshotName, false));
+  public void testExportRetry() throws Exception {
+    Path copyDir = getLocalDestinationDir();
+    FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
+    copyDir = copyDir.makeQualified(fs);
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true);
+    conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 2);
+    conf.setInt("mapreduce.map.maxattempts", 3);
+    testExportFileSystemState(conf, tableName, snapshotName, snapshotName, tableNumFiles,
+        TEST_UTIL.getDefaultRootDirPath(), copyDir, true, getBypassRegionPredicate(), true);
   }
 
   /**
-   * Check that ExportSnapshot will succede if something fails but the retry succede.
+   * Check that ExportSnapshot will fail if we inject failure more times than MR will retry.
    */
   @Test
-  public void testExportRetry() throws Exception {
-    assertEquals(0, runExportAndInjectFailures(snapshotName, true));
-  }
-
-  /*
-   * Execute the ExportSnapshot job injecting failures
-   */
-  private int runExportAndInjectFailures(final byte[] snapshotName, boolean retry)
-      throws Exception {
+  public void testExportFailure() throws Exception {
     Path copyDir = getLocalDestinationDir();
-    URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri();
     FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
     copyDir = copyDir.makeQualified(fs);
-
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
-    conf.setBoolean(ExportSnapshot.CONF_TEST_FAILURE, true);
-    conf.setBoolean(ExportSnapshot.CONF_TEST_RETRY, retry);
-    if (!retry) {
-      conf.setInt("mapreduce.map.maxattempts", 3);
-    }
-    // Export Snapshot
-    Path sourceDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
-    String[] args = new String[] { "--snapshot", Bytes.toString(snapshotName),
-        "--copy-from", sourceDir.toString(), "--copy-to", copyDir.toString() };
-    return ToolRunner.run(conf, new ExportSnapshot(), args);
+    conf.setBoolean(ExportSnapshot.Testing.CONF_TEST_FAILURE, true);
+    conf.setInt(ExportSnapshot.Testing.CONF_TEST_FAILURE_COUNT, 4);
+    conf.setInt("mapreduce.map.maxattempts", 3);
+    testExportFileSystemState(conf, tableName, snapshotName, snapshotName, tableNumFiles,
+        TEST_UTIL.getDefaultRootDirPath(), copyDir, true, getBypassRegionPredicate(), false);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
index e2d7c11..cd5ff6c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
@@ -104,7 +104,7 @@ public class TestExportSnapshotNoCluster {
     TableName tableName = builder.getTableDescriptor().getTableName();
     TestExportSnapshot.testExportFileSystemState(TEST_UTIL.getConfiguration(),
       tableName, snapshotName, snapshotName, snapshotFilesCount,
-      testDir, getDestinationDir(), false, null);
+      testDir, getDestinationDir(), false, null, true);
   }
 
   private Path getDestinationDir() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
index c375b0a..55686b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobExportSnapshot.java
@@ -45,7 +45,12 @@ public class TestMobExportSnapshot extends TestExportSnapshot {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
-    TEST_UTIL.startMiniCluster(3);
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
index 8154995..c0f31d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobSecureExportSnapshot.java
@@ -37,6 +37,9 @@ public class TestMobSecureExportSnapshot extends TestMobExportSnapshot {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
 
     // set the always on security provider
     UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
@@ -45,7 +48,9 @@ public class TestMobSecureExportSnapshot extends TestMobExportSnapshot {
     // setup configuration
     SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
 
-    TEST_UTIL.startMiniCluster(3);
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
 
     // Wait for the ACL table to become available
     TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/da5fb27e/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
index f335e4e..4d35b71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSecureExportSnapshot.java
@@ -42,6 +42,9 @@ public class TestSecureExportSnapshot extends TestExportSnapshot {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     setUpBaseConf(TEST_UTIL.getConfiguration());
+    // Setup separate test-data directory for MR cluster and set corresponding configurations.
+    // Otherwise, different test classes running MR cluster can step on each other.
+    TEST_UTIL.getDataTestDir();
 
     // set the always on security provider
     UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
@@ -50,7 +53,9 @@ public class TestSecureExportSnapshot extends TestExportSnapshot {
     // setup configuration
     SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
 
-    TEST_UTIL.startMiniCluster(3);
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniMapReduceCluster();
+    TEST_UTIL.startMiniHBaseCluster(1, 3);
 
     // Wait for the ACL table to become available
     TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);


[02/50] [abbrv] hbase git commit: HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index bb6b40e..0071bef 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -440,7 +440,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -1245,7 +1245,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -1867,7 +1867,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -2577,7 +2577,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -3220,7 +3220,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -4025,7 +4025,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -4582,7 +4582,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -5250,7 +5250,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5843,7 +5843,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getRegionCount() > 0) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegionList().hashCode();
@@ -6698,7 +6698,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -7190,7 +7190,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -7713,7 +7713,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -8197,7 +8197,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -8766,7 +8766,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -9208,7 +9208,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -9731,7 +9731,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -10318,7 +10318,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableSchema()) {
         hash = (37 * hash) + TABLE_SCHEMA_FIELD_NUMBER;
         hash = (53 * hash) + getTableSchema().hashCode();
@@ -11070,7 +11070,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -11646,7 +11646,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -12305,7 +12305,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -12923,7 +12923,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLENAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -13628,7 +13628,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -14204,7 +14204,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -14863,7 +14863,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -15439,7 +15439,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -16098,7 +16098,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -16741,7 +16741,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -17546,7 +17546,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -18122,7 +18122,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceDescriptor()) {
         hash = (37 * hash) + NAMESPACEDESCRIPTOR_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceDescriptor().hashCode();
@@ -18781,7 +18781,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -19368,7 +19368,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceName()) {
         hash = (37 * hash) + NAMESPACENAME_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceName().hashCode();
@@ -19975,7 +19975,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -20551,7 +20551,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceDescriptor()) {
         hash = (37 * hash) + NAMESPACEDESCRIPTOR_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceDescriptor().hashCode();
@@ -21210,7 +21210,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -21713,7 +21713,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceName()) {
         hash = (37 * hash) + NAMESPACENAME_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceName().hashCode();
@@ -22253,7 +22253,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceDescriptor()) {
         hash = (37 * hash) + NAMESPACEDESCRIPTOR_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceDescriptor().hashCode();
@@ -22776,7 +22776,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -23237,7 +23237,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getNamespaceDescriptorCount() > 0) {
         hash = (37 * hash) + NAMESPACEDESCRIPTOR_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceDescriptorList().hashCode();
@@ -23983,7 +23983,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceName()) {
         hash = (37 * hash) + NAMESPACENAME_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceName().hashCode();
@@ -24542,7 +24542,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getTableSchemaCount() > 0) {
         hash = (37 * hash) + TABLESCHEMA_FIELD_NUMBER;
         hash = (53 * hash) + getTableSchemaList().hashCode();
@@ -25288,7 +25288,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNamespaceName()) {
         hash = (37 * hash) + NAMESPACENAME_FIELD_NUMBER;
         hash = (53 * hash) + getNamespaceName().hashCode();
@@ -25847,7 +25847,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getTableNameCount() > 0) {
         hash = (37 * hash) + TABLENAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableNameList().hashCode();
@@ -26513,7 +26513,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -26886,7 +26886,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -27259,7 +27259,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -27632,7 +27632,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -28005,7 +28005,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -28426,7 +28426,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasInMaintenanceMode()) {
         hash = (37 * hash) + INMAINTENANCEMODE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -28896,7 +28896,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasForce()) {
         hash = (37 * hash) + FORCE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -29367,7 +29367,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBalancerRan()) {
         hash = (37 * hash) + BALANCER_RAN_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -29883,7 +29883,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasOn()) {
         hash = (37 * hash) + ON_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -30399,7 +30399,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPrevBalanceValue()) {
         hash = (37 * hash) + PREV_BALANCE_VALUE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -30822,7 +30822,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -31243,7 +31243,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -31853,7 +31853,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -32492,7 +32492,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getPrevValueCount() > 0) {
         hash = (37 * hash) + PREV_VALUE_FIELD_NUMBER;
         hash = (53 * hash) + getPrevValueList().hashCode();
@@ -33008,7 +33008,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSwitchType()) {
         hash = (37 * hash) + SWITCH_TYPE_FIELD_NUMBER;
         hash = (53 * hash) + switchType_;
@@ -33485,7 +33485,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -33911,7 +33911,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -34332,7 +34332,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNormalizerRan()) {
         hash = (37 * hash) + NORMALIZER_RAN_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -34806,7 +34806,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasOn()) {
         hash = (37 * hash) + ON_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -35276,7 +35276,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPrevNormalizerValue()) {
         hash = (37 * hash) + PREV_NORMALIZER_VALUE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -35699,7 +35699,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -36120,7 +36120,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -36546,7 +36546,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -36963,7 +36963,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasScanResult()) {
         hash = (37 * hash) + SCAN_RESULT_FIELD_NUMBER;
         hash = (53 * hash) + getScanResult();
@@ -37433,7 +37433,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEnable()) {
         hash = (37 * hash) + ENABLE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -37903,7 +37903,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPrevValue()) {
         hash = (37 * hash) + PREV_VALUE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -38326,7 +38326,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -38747,7 +38747,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasValue()) {
         hash = (37 * hash) + VALUE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -39173,7 +39173,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -39594,7 +39594,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCleanerChoreRan()) {
         hash = (37 * hash) + CLEANER_CHORE_RAN_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -40068,7 +40068,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasOn()) {
         hash = (37 * hash) + ON_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -40538,7 +40538,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPrevValue()) {
         hash = (37 * hash) + PREV_VALUE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -40961,7 +40961,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -41382,7 +41382,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasValue()) {
         hash = (37 * hash) + VALUE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -41877,7 +41877,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSnapshot()) {
         hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
         hash = (53 * hash) + getSnapshot().hashCode();
@@ -42448,7 +42448,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasExpectedTimeout()) {
         hash = (37 * hash) + EXPECTED_TIMEOUT_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -42874,7 +42874,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -43335,7 +43335,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getSnapshotsCount() > 0) {
         hash = (37 * hash) + SNAPSHOTS_FIELD_NUMBER;
         hash = (53 * hash) + getSnapshotsList().hashCode();
@@ -44070,7 +44070,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSnapshot()) {
         hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
         hash = (53 * hash) + getSnapshot().hashCode();
@@ -44593,7 +44593,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -45119,7 +45119,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSnapshot()) {
         hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
         hash = (53 * hash) + getSnapshot().hashCode();
@@ -45782,7 +45782,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -46280,7 +46280,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSnapshot()) {
         hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
         hash = (53 * hash) + getSnapshot().hashCode();
@@ -46916,7 +46916,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDone()) {
         hash = (37 * hash) + DONE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -47551,7 +47551,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasSnapshot()) {
         hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
         hash = (53 * hash) + getSnapshot().hashCode();
@@ -48117,7 +48117,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDone()) {
         hash = (37 * hash) + DONE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -48609,7 +48609,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -49218,7 +49218,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasYetToUpdateRegions()) {
         hash = (37 * hash) + YET_TO_UPDATE_REGIONS_FIELD_NUMBER;
         hash = (53 * hash) + getYetToUpdateRegions();
@@ -49964,7 +49964,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getTableNamesCount() > 0) {
         hash = (37 * hash) + TABLE_NAMES_FIELD_NUMBER;
         hash = (53 * hash) + getTableNamesList().hashCode();
@@ -50948,7 +50948,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getTableSchemaCount() > 0) {
         hash = (37 * hash) + TABLE_SCHEMA_FIELD_NUMBER;
         hash = (53 * hash) + getTableSchemaList().hashCode();
@@ -51806,7 +51806,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegex()) {
         hash = (37 * hash) + REGEX_FIELD_NUMBER;
         hash = (53 * hash) + getRegex().hashCode();
@@ -52499,7 +52499,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getTableNamesCount() > 0) {
         hash = (37 * hash) + TABLE_NAMES_FIELD_NUMBER;
         hash = (53 * hash) + getTableNamesList().hashCode();
@@ -53234,7 +53234,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -53826,7 +53826,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableState()) {
         hash = (37 * hash) + TABLE_STATE_FIELD_NUMBER;
         hash = (53 * hash) + getTableState().hashCode();
@@ -54349,7 +54349,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -54791,7 +54791,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasClusterStatus()) {
         hash = (37 * hash) + CLUSTER_STATUS_FIELD_NUMBER;
         hash = (53 * hash) + getClusterStatus().hashCode();
@@ -55314,7 +55314,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -55735,7 +55735,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasIsMasterRunning()) {
         hash = (37 * hash) + IS_MASTER_RUNNING_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -56230,7 +56230,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcedure()) {
         hash = (37 * hash) + PROCEDURE_FIELD_NUMBER;
         hash = (53 * hash) + getProcedure().hashCode();
@@ -56839,7 +56839,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasExpectedTimeout()) {
         hash = (37 * hash) + EXPECTED_TIMEOUT_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -57377,7 +57377,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcedure()) {
         hash = (37 * hash) + PROCEDURE_FIELD_NUMBER;
         hash = (53 * hash) + getProcedure().hashCode();
@@ -58008,7 +58008,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasDone()) {
         hash = (37 * hash) + DONE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -58624,7 +58624,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -59388,7 +59388,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasState()) {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + state_;
@@ -60187,7 +60187,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -60707,7 +60707,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasIsProcedureAborted()) {
         hash = (37 * hash) + IS_PROCEDURE_ABORTED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -61133,7 +61133,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -61594,7 +61594,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getProcedureCount() > 0) {
         hash = (37 * hash) + PROCEDURE_FIELD_NUMBER;
         hash = (53 * hash) + getProcedureList().hashCode();
@@ -62698,7 +62698,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserName()) {
         hash = (37 * hash) + USER_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getUserName().hashCode();
@@ -63730,7 +63730,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -64172,7 +64172,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -64764,7 +64764,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
         hash = (53 * hash) + getRegion().hashCode();
@@ -65335,7 +65335,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCompactionTimestamp()) {
         hash = (37 * hash) + COMPACTION_TIMESTAMP_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -65761,7 +65761,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -66346,7 +66346,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getCapabilitiesCount() > 0) {
         hash = (37 * hash) + CAPABILITIES_FIELD_NUMBER;
         hash = (53 * hash) + capabilities_.hashCode();
@@ -66816,7 +66816,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -67277,7 +67277,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getServerNameCount() > 0) {
         hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getServerNameList().hashCode();
@@ -68031,7 +68031,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getServerNameCount() > 0) {
         hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getServerNameList().hashCode();
@@ -68697,7 +68697,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -69158,7 +69158,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getServerNameCount() > 0) {
         hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getServerNameList().hashCode();
@@ -69824,7 +69824,7 @@ public final class MasterProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
index 91989d7..b0ba843 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ProcedureProtos.java
@@ -1190,7 +1190,7 @@ public final class ProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasClassName()) {
         hash = (37 * hash) + CLASS_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getClassName().hashCode();
@@ -2683,7 +2683,7 @@ public final class ProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasExecuted()) {
         hash = (37 * hash) + EXECUTED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -3194,7 +3194,7 @@ public final class ProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getStateCount() > 0) {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + getStateList().hashCode();
@@ -3852,7 +3852,7 @@ public final class ProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasVersion()) {
         hash = (37 * hash) + VERSION_FIELD_NUMBER;
         hash = (53 * hash) + getVersion();
@@ -4527,7 +4527,7 @@ public final class ProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasVersion()) {
         hash = (37 * hash) + VERSION_FIELD_NUMBER;
         hash = (53 * hash) + getVersion();
@@ -5309,7 +5309,7 @@ public final class ProcedureProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasStartId()) {
           hash = (37 * hash) + START_ID_FIELD_NUMBER;
           hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -5890,7 +5890,7 @@ public final class ProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getNodeCount() > 0) {
         hash = (37 * hash) + NODE_FIELD_NUMBER;
         hash = (53 * hash) + getNodeList().hashCode();
@@ -6939,7 +6939,7 @@ public final class ProcedureProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasType()) {
         hash = (37 * hash) + TYPE_FIELD_NUMBER;
         hash = (53 * hash) + type_;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index d14336a..01ba8f6 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -615,7 +615,7 @@ public final class QuotaProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTimeUnit()) {
         hash = (37 * hash) + TIME_UNIT_FIELD_NUMBER;
         hash = (53 * hash) + timeUnit_;
@@ -1577,7 +1577,7 @@ public final class QuotaProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasReqNum()) {
         hash = (37 * hash) + REQ_NUM_FIELD_NUMBER;
         hash = (53 * hash) + getReqNum().hashCode();
@@ -2939,7 +2939,7 @@ public final class QuotaProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasType()) {
         hash = (37 * hash) + TYPE_FIELD_NUMBER;
         hash = (53 * hash) + type_;
@@ -3619,7 +3619,7 @@ public final class QuotaProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasBypassGlobals()) {
         hash = (37 * hash) + BYPASS_GLOBALS_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -4187,7 +4187,7 @@ public final class QuotaProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java
index d12b30e..a7adb06 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RPCProtos.java
@@ -288,7 +288,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEffectiveUser()) {
         hash = (37 * hash) + EFFECTIVE_USER_FIELD_NUMBER;
         hash = (53 * hash) + getEffectiveUser().hashCode();
@@ -1370,7 +1370,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasUserInfo()) {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
@@ -2584,7 +2584,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCryptoCipherMeta()) {
         hash = (37 * hash) + CRYPTO_CIPHER_META_FIELD_NUMBER;
         hash = (53 * hash) + getCryptoCipherMeta().hashCode();
@@ -3210,7 +3210,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLength()) {
         hash = (37 * hash) + LENGTH_FIELD_NUMBER;
         hash = (53 * hash) + getLength();
@@ -4059,7 +4059,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasExceptionClassName()) {
         hash = (37 * hash) + EXCEPTION_CLASS_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getExceptionClassName().hashCode();
@@ -5152,7 +5152,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTransformation()) {
         hash = (37 * hash) + TRANSFORMATION_FIELD_NUMBER;
         hash = (53 * hash) + getTransformation().hashCode();
@@ -6262,7 +6262,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCallId()) {
         hash = (37 * hash) + CALL_ID_FIELD_NUMBER;
         hash = (53 * hash) + getCallId();
@@ -7493,7 +7493,7 @@ public final class RPCProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasCallId()) {
         hash = (37 * hash) + CALL_ID_FIELD_NUMBER;
         hash = (53 * hash) + getCallId();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java
index 0a58113..2421f0a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionNormalizerProtos.java
@@ -174,7 +174,7 @@ public final class RegionNormalizerProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasNormalizerOn()) {
         hash = (37 * hash) + NORMALIZER_ON_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
index 8f368e9..d7d4db0 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
@@ -416,7 +416,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPort()) {
         hash = (37 * hash) + PORT_FIELD_NUMBER;
         hash = (53 * hash) + getPort();
@@ -1260,7 +1260,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getMapEntriesCount() > 0) {
         hash = (37 * hash) + MAP_ENTRIES_FIELD_NUMBER;
         hash = (53 * hash) + getMapEntriesList().hashCode();
@@ -2210,7 +2210,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServer()) {
         hash = (37 * hash) + SERVER_FIELD_NUMBER;
         hash = (53 * hash) + getServer().hashCode();
@@ -2914,7 +2914,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3482,7 +3482,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServer()) {
         hash = (37 * hash) + SERVER_FIELD_NUMBER;
         hash = (53 * hash) + getServer().hashCode();
@@ -4159,7 +4159,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4596,7 +4596,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegionName()) {
         hash = (37 * hash) + REGION_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getRegionName().hashCode();
@@ -5231,7 +5231,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasLastFlushedSequenceId()) {
         hash = (37 * hash) + LAST_FLUSHED_SEQUENCE_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -6462,7 +6462,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTransitionCode()) {
         hash = (37 * hash) + TRANSITION_CODE_FIELD_NUMBER;
         hash = (53 * hash) + transitionCode_;
@@ -7496,7 +7496,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServer()) {
         hash = (37 * hash) + SERVER_FIELD_NUMBER;
         hash = (53 * hash) + getServer().hashCode();
@@ -8446,7 +8446,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasErrorMessage()) {
         hash = (37 * hash) + ERROR_MESSAGE_FIELD_NUMBER;
         hash = (53 * hash) + getErrorMessage().hashCode();
@@ -9142,7 +9142,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegionInfo()) {
         hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getRegionInfo().hashCode();
@@ -9857,7 +9857,7 @@ public final class RegionServerStatusProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasProcId()) {
         hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
index 6a6d4bf..4052c12 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
@@ -258,7 +258,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -1426,7 +1426,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasClusterkey()) {
         hash = (37 * hash) + CLUSTERKEY_FIELD_NUMBER;
         hash = (53 * hash) + getClusterkey().hashCode();
@@ -3184,7 +3184,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasState()) {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + state_;
@@ -3837,7 +3837,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasId()) {
         hash = (37 * hash) + ID_FIELD_NUMBER;
         hash = (53 * hash) + getId().hashCode();
@@ -4658,7 +4658,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPosition()) {
         hash = (37 * hash) + POSITION_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
@@ -5236,7 +5236,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerId()) {
         hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerId().hashCode();
@@ -5853,7 +5853,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -6306,7 +6306,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerId()) {
         hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerId().hashCode();
@@ -6777,7 +6777,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -7230,7 +7230,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerId()) {
         hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerId().hashCode();
@@ -7701,7 +7701,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -8154,7 +8154,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerId()) {
         hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerId().hashCode();
@@ -8625,7 +8625,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -9078,7 +9078,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerId()) {
         hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerId().hashCode();
@@ -9696,7 +9696,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerId()) {
         hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerId().hashCode();
@@ -10460,7 +10460,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasPeerId()) {
         hash = (37 * hash) + PEER_ID_FIELD_NUMBER;
         hash = (53 * hash) + getPeerId().hashCode();
@@ -11077,7 +11077,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -11526,7 +11526,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasRegex()) {
         hash = (37 * hash) + REGEX_FIELD_NUMBER;
         hash = (53 * hash) + getRegex().hashCode();
@@ -12082,7 +12082,7 @@ public final class ReplicationProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (getPeerDescCount() > 0) {
         hash = (37 * hash) + PEER_DESC_FIELD_NUMBER;
         hash = (53 * hash) + getPeerDescList().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
index d40086b..a366f77 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
@@ -496,7 +496,7 @@ public final class SnapshotProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasType()) {
         hash = (37 * hash) + TYPE_FIELD_NUMBER;
         hash = (53 * hash) + type_;
@@ -1549,7 +1549,7 @@ public final class SnapshotProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasName()) {
           hash = (37 * hash) + NAME_FIELD_NUMBER;
           hash = (53 * hash) + getName().hashCode();
@@ -2362,7 +2362,7 @@ public final class SnapshotProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasFamilyName()) {
           hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
           hash = (53 * hash) + getFamilyName().hashCode();
@@ -3131,7 +3131,7 @@ public final class SnapshotProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasVersion()) {
         hash = (37 * hash) + VERSION_FIELD_NUMBER;
         hash = (53 * hash) + getVersion();
@@ -4146,7 +4146,7 @@ public final class SnapshotProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableSchema()) {
         hash = (37 * hash) + TABLE_SCHEMA_FIELD_NUMBER;
         hash = (53 * hash) + getTableSchema().hashCode();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java
index 755bf65..f4897f3 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/TracingProtos.java
@@ -224,7 +224,7 @@ public final class TracingProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTraceId()) {
         hash = (37 * hash) + TRACE_ID_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java
index db73e41..048f4b1 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/WALProtos.java
@@ -505,7 +505,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasHasCompression()) {
         hash = (37 * hash) + HAS_COMPRESSION_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -1894,7 +1894,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEncodedRegionName()) {
         hash = (37 * hash) + ENCODED_REGION_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getEncodedRegionName().hashCode();
@@ -3639,7 +3639,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFamily()) {
         hash = (37 * hash) + FAMILY_FIELD_NUMBER;
         hash = (53 * hash) + getFamily().hashCode();
@@ -4623,7 +4623,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -6196,7 +6196,7 @@ public final class WALProtos {
           return memoizedHashCode;
         }
         int hash = 41;
-        hash = (19 * hash) + getDescriptorForType().hashCode();
+        hash = (19 * hash) + getDescriptor().hashCode();
         if (hasFamilyName()) {
           hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
           hash = (53 * hash) + getFamilyName().hashCode();
@@ -7029,7 +7029,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasAction()) {
         hash = (37 * hash) + ACTION_FIELD_NUMBER;
         hash = (53 * hash) + action_;
@@ -8280,7 +8280,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasFamilyName()) {
         hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getFamilyName().hashCode();
@@ -9291,7 +9291,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasTableName()) {
         hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
         hash = (53 * hash) + getTableName().hashCode();
@@ -10727,7 +10727,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEventType()) {
         hash = (37 * hash) + EVENT_TYPE_FIELD_NUMBER;
         hash = (53 * hash) + eventType_;
@@ -11853,7 +11853,7 @@ public final class WALProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
index caafc01..f685c6b 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
@@ -364,7 +364,7 @@ public final class ZooKeeperProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasServer()) {
         hash = (37 * hash) + SERVER_FIELD_NUMBER;
         hash = (53 * hash) + getServer().hashCode();
@@ -1269,7 +1269,7 @@ public final class ZooKeeperProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasMaster()) {
         hash = (37 * hash) + MASTER_FIELD_NUMBER;
         hash = (53 * hash) + getMaster().hashCode();
@@ -2054,7 +2054,7 @@ public final class ZooKeeperProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasStartDate()) {
         hash = (37 * hash) + START_DATE_FIELD_NUMBER;
         hash = (53 * hash) + getStartDate().hashCode();
@@ -2951,7 +2951,7 @@ public final class ZooKeeperProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasState()) {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + state_;
@@ -3777,7 +3777,7 @@ public final class ZooKeeperProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasState()) {
         hash = (37 * hash) + STATE_FIELD_NUMBER;
         hash = (53 * hash) + state_;
@@ -4285,7 +4285,7 @@ public final class ZooKeeperProtos {
         return memoizedHashCode;
       }
       int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (19 * hash) + getDescriptor().hashCode();
       if (hasEnabled()) {
         hash = (37 * hash) + ENABLED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(


[25/50] [abbrv] hbase git commit: HBASE-17881 Remove the ByteBufferCellImpl

Posted by sy...@apache.org.
HBASE-17881 Remove the ByteBufferCellImpl


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

Branch: refs/heads/hbase-12439
Commit: 18c5ecf6ed57e80b32568ca1a1a12c7af36bab46
Parents: 1a701ce
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Wed Apr 5 21:11:29 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Apr 7 21:14:19 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/TestComparators.java    |  14 +-
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java  |   4 +-
 .../apache/hadoop/hbase/TestCellComparator.java |   7 +-
 .../org/apache/hadoop/hbase/TestCellUtil.java   | 184 +------------------
 .../filter/TestSingleColumnValueFilter.java     |  36 ++--
 5 files changed, 36 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
index d9e4033..0c69ece 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
@@ -21,11 +21,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -51,18 +51,18 @@ public class TestComparators {
     // Row compare
     KeyValue kv = new KeyValue(r1, f, q1, v1);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     ByteArrayComparable comparable = new BinaryComparator(r1);
     assertEquals(0, CellComparator.compareRow(bbCell, comparable));
     assertEquals(0, CellComparator.compareRow(kv, comparable));
     kv = new KeyValue(r0, f, q1, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareRow(bbCell, comparable) > 0);
     assertTrue(CellComparator.compareRow(kv, comparable) > 0);
     kv = new KeyValue(r2, f, q1, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareRow(bbCell, comparable) < 0);
     assertTrue(CellComparator.compareRow(kv, comparable) < 0);
     // Qualifier compare
@@ -71,12 +71,12 @@ public class TestComparators {
     assertEquals(0, CellComparator.compareQualifier(kv, comparable));
     kv = new KeyValue(r2, f, q2, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertEquals(0, CellComparator.compareQualifier(bbCell, comparable));
     assertEquals(0, CellComparator.compareQualifier(kv, comparable));
     kv = new KeyValue(r2, f, q3, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareQualifier(bbCell, comparable) < 0);
     assertTrue(CellComparator.compareQualifier(kv, comparable) < 0);
     // Value compare
@@ -85,7 +85,7 @@ public class TestComparators {
     assertEquals(0, CellComparator.compareValue(kv, comparable));
     kv = new KeyValue(r1, f, q1, v2);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareValue(bbCell, comparable) < 0);
     assertTrue(CellComparator.compareValue(kv, comparable) < 0);
     // Family compare

http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
index f22e5d4..33e3cd9 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
@@ -23,12 +23,12 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferCell;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -71,7 +71,7 @@ public class TestKeyOnlyFilter {
         v.length, tags);
 
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    ByteBufferCellImpl bbCell = new ByteBufferCellImpl(buffer, 0,
+    ByteBufferKeyValue bbCell = new ByteBufferKeyValue(buffer, 0,
         buffer.remaining());
 
     // KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>

http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
index 8740cfb..791b9ee 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -102,15 +101,15 @@ public class TestCellComparator {
     byte[] v = Bytes.toBytes("val1");
     KeyValue kv = new KeyValue(r1, f1, q1, v);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell1 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell1 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     kv = new KeyValue(r2, f1, q1, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell2 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell2 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertEquals(0, CellComparator.compareColumns(bbCell1, bbCell2));
     assertEquals(0, CellComparator.compareColumns(bbCell1, kv));
     kv = new KeyValue(r2, f1, q2, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell3 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell3 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertEquals(0, CellComparator.compareFamilies(bbCell2, bbCell3));
     assertTrue(CellComparator.compareQualifiers(bbCell2, bbCell3) < 0);
     assertTrue(CellComparator.compareColumns(bbCell2, bbCell3) < 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
index 441d1b5..bbc7a31 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
@@ -33,7 +33,6 @@ import java.util.TreeMap;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -416,7 +415,7 @@ public class TestCellUtil {
     byte[] tags = Bytes.toBytes("tag1");
     KeyValue kv = new KeyValue(r, f, q, 0, q.length, 1234L, Type.Put, v, 0, v.length, tags);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     byte[] rDest = CellUtil.cloneRow(bbCell);
     assertTrue(Bytes.equals(r, rDest));
     byte[] fDest = CellUtil.cloneFamily(bbCell);
@@ -440,10 +439,10 @@ public class TestCellUtil {
     byte[] tags = Bytes.toBytes("tag1");
     KeyValue kv = new KeyValue(r, f, q1, 0, q1.length, 1234L, Type.Put, v, 0, v.length, tags);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell1 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell1 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     kv = new KeyValue(r, f, q2, 0, q2.length, 1234L, Type.Put, v, 0, v.length, tags);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell2 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell2 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertTrue(CellUtil.matchingRows(bbCell1, bbCell2));
     assertTrue(CellUtil.matchingRows(kv, bbCell2));
     assertTrue(CellUtil.matchingRow(bbCell1, r));
@@ -473,191 +472,20 @@ public class TestCellUtil {
     byte[] v = Bytes.toBytes(vl);
     KeyValue kv = new KeyValue(r, f, q, v);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertEquals(ri, CellUtil.getRowAsInt(bbCell));
     assertEquals(vl, CellUtil.getValueAsLong(bbCell));
     double vd = 3005.5;
     v = Bytes.toBytes(vd);
     kv = new KeyValue(r, f, q, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertEquals(vd, CellUtil.getValueAsDouble(bbCell), 0.0);
     BigDecimal bd = new BigDecimal(9999);
     v = Bytes.toBytes(bd);
     kv = new KeyValue(r, f, q, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
   }
-
-  // TODO remove this test impl once we have a Cell implementation backed by ByteBuffer
-  public static class ByteBufferCellImpl extends ByteBufferCell {
-
-    private final ByteBuffer buffer;
-    private final int offset, length;
-
-    public ByteBufferCellImpl(ByteBuffer buffer, int offset, int length) {
-      this.buffer = buffer;
-      this.offset = offset;
-      this.length = length;
-    }
-
-    @Override
-    public byte[] getRowArray() {
-      return CellUtil.cloneRow(this);
-    }
-
-    @Override
-    public int getRowOffset() {
-      return 0;
-    }
-
-    @Override
-    public short getRowLength() {
-      return ByteBufferUtils.toShort(this.buffer, this.offset + KeyValue.ROW_OFFSET);
-    }
-
-    @Override
-    public byte[] getFamilyArray() {
-      return CellUtil.cloneFamily(this);
-    }
-
-    @Override
-    public int getFamilyOffset() {
-      return 0;
-    }
-
-    @Override
-    public byte getFamilyLength() {
-      return this.buffer.get(getFamilyPosition() - 1);
-    }
-
-    @Override
-    public byte[] getQualifierArray() {
-      return CellUtil.cloneQualifier(this);
-    }
-
-    @Override
-    public int getQualifierOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getQualifierLength() {
-      return getKeyLength()
-          - (int) KeyValue.getKeyDataStructureSize(getRowLength(), getFamilyLength(), 0);
-    }
-
-    private int getKeyLength() {
-      return ByteBufferUtils.toInt(this.buffer, this.offset);
-    }
-
-    @Override
-    public long getTimestamp() {
-      int tsOffset = this.offset + KeyValue.ROW_OFFSET + getKeyLength()
-          - KeyValue.TIMESTAMP_TYPE_SIZE;
-      return ByteBufferUtils.toLong(buffer, tsOffset);
-    }
-
-    @Override
-    public byte getTypeByte() {
-      return KeyValue.Type.Put.getCode();
-    }
-
-    @Override
-    public long getSequenceId() {
-      return 0;
-    }
-
-    @Override
-    public byte[] getValueArray() {
-      return CellUtil.cloneValue(this);
-    }
-
-    @Override
-    public int getValueOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getValueLength() {
-      return ByteBufferUtils.toInt(this.buffer, this.offset + KeyValue.KEY_LENGTH_SIZE);
-    }
-
-    @Override
-    public byte[] getTagsArray() {
-      byte[] tDest = new byte[getTagsLength()];
-      CellUtil.copyTagTo(this, tDest, 0);
-      return tDest;
-    }
-
-    @Override
-    public int getTagsOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getTagsLength() {
-      int tagsLen = this.length
-          - (getKeyLength() + getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
-      if (tagsLen > 0) {
-        tagsLen -= KeyValue.TAGS_LENGTH_SIZE;
-      }
-      return tagsLen;
-    }
-
-    @Override
-    public ByteBuffer getRowByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getRowPosition() {
-      return this.offset + KeyValue.ROW_KEY_OFFSET;
-    }
-
-    @Override
-    public ByteBuffer getFamilyByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getFamilyPosition() {
-      return this.offset + KeyValue.ROW_KEY_OFFSET + getRowLength() + Bytes.SIZEOF_BYTE;
-    }
-
-    @Override
-    public ByteBuffer getQualifierByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getQualifierPosition() {
-      return getFamilyPosition() + getFamilyLength();
-    }
-
-    @Override
-    public ByteBuffer getValueByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getValuePosition() {
-      return this.offset + KeyValue.ROW_OFFSET + getKeyLength();
-    }
-
-    @Override
-    public ByteBuffer getTagsByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getTagsPosition() {
-      int tagsLen = getTagsLength();
-      if (tagsLen == 0) {
-        return this.offset + this.length;
-      }
-      return this.offset + this.length - tagsLen;
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
index 48a5b6f..134e8d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
@@ -24,10 +24,10 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.regex.Pattern;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -108,7 +108,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("less than", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
 
@@ -117,7 +117,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("Equals 100", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
 
@@ -126,7 +126,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     filter.reset();
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("include 120", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
   }
 
@@ -135,29 +135,29 @@ public class TestSingleColumnValueFilter {
     KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
     assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
     assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
     assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("basicFilterNotNull", filter.filterRow());
     filter.reset();
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
     assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
     assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
     assertTrue("basicFilterNotNull", filter.filterRow());
@@ -166,12 +166,12 @@ public class TestSingleColumnValueFilter {
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
     assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
     assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("basicFilterNotNull", filter.filterRow());
   }
@@ -181,14 +181,14 @@ public class TestSingleColumnValueFilter {
     KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
     assertTrue("null1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("null1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("null1FilterRow", filter.filterRow());
     filter.reset();
     kv = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
     assertTrue("null2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("null2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertTrue("null2FilterRow", filter.filterRow());
   }
@@ -200,13 +200,13 @@ public class TestSingleColumnValueFilter {
     assertTrue("substrTrue",
       filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("substrTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_2);
     assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("substrFilterNotNull", filter.filterRow());
@@ -219,13 +219,13 @@ public class TestSingleColumnValueFilter {
     assertTrue("regexTrue",
       filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_2);
     assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("regexFilterNotNull", filter.filterRow());
@@ -238,7 +238,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("regexTrue",
       filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("regexFilterNotNull", filter.filterRow());


[22/50] [abbrv] hbase git commit: HBASE-17869 UnsafeAvailChecker wrongly returns false on ppc

Posted by sy...@apache.org.
HBASE-17869 UnsafeAvailChecker wrongly returns false on ppc


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

Branch: refs/heads/hbase-12439
Commit: af604f0c0cf3c40c56746150ffa860aad07f128a
Parents: 9109803
Author: Jerry He <je...@apache.org>
Authored: Thu Apr 6 16:04:47 2017 -0700
Committer: Jerry He <je...@apache.org>
Committed: Thu Apr 6 16:04:47 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/util/UnsafeAvailChecker.java   | 24 ++++++++++++--------
 1 file changed, 15 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/af604f0c/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAvailChecker.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAvailChecker.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAvailChecker.java
index 90e6ec8..886cb3c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAvailChecker.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAvailChecker.java
@@ -51,15 +51,21 @@ public class UnsafeAvailChecker {
     });
     // When Unsafe itself is not available/accessible consider unaligned as false.
     if (avail) {
-      try {
-        // Using java.nio.Bits#unaligned() to check for unaligned-access capability
-        Class<?> clazz = Class.forName("java.nio.Bits");
-        Method m = clazz.getDeclaredMethod("unaligned");
-        m.setAccessible(true);
-        unaligned = (Boolean) m.invoke(null);
-      } catch (Exception e) {
-        LOG.warn("java.nio.Bits#unaligned() check failed."
-            + "Unsafe based read/write of primitive types won't be used", e);
+      String arch = System.getProperty("os.arch");
+      if ("ppc64".equals(arch) || "ppc64le".equals(arch)) {
+        // java.nio.Bits.unaligned() wrongly returns false on ppc (JDK-8165231),
+        unaligned = true;
+      } else {
+        try {
+          // Using java.nio.Bits#unaligned() to check for unaligned-access capability
+          Class<?> clazz = Class.forName("java.nio.Bits");
+          Method m = clazz.getDeclaredMethod("unaligned");
+          m.setAccessible(true);
+          unaligned = (Boolean) m.invoke(null);
+        } catch (Exception e) {
+          LOG.warn("java.nio.Bits#unaligned() check failed."
+              + "Unsafe based read/write of primitive types won't be used", e);
+        }
       }
     }
   }


[08/50] [abbrv] hbase git commit: HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs

Posted by sy...@apache.org.
HBASE-16780 Since move to protobuf3.1, Cells are limited to 64MB where previous they had no limit Update internal pb to 3.2 from 3.1.; AMENDMENT -- FORGOT TO REBUILD PBs


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

Branch: refs/heads/hbase-12439
Commit: e916b79db58bb9be806a833b2c0e675f1136c15a
Parents: 73e1bcd
Author: Michael Stack <st...@apache.org>
Authored: Mon Apr 3 15:26:11 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Apr 3 15:26:11 2017 -0700

----------------------------------------------------------------------
 .../google/protobuf/AbstractMessageLite.java    |    1 -
 .../com/google/protobuf/AbstractParser.java     |    2 +-
 .../hbase/shaded/com/google/protobuf/Any.java   |   26 +-
 .../com/google/protobuf/AnyOrBuilder.java       |    6 +-
 .../hbase/shaded/com/google/protobuf/Api.java   |   68 +-
 .../com/google/protobuf/ApiOrBuilder.java       |   18 +-
 .../shaded/com/google/protobuf/ApiProto.java    |    7 +-
 .../shaded/com/google/protobuf/BoolValue.java   |   10 +-
 .../com/google/protobuf/BoolValueOrBuilder.java |    2 +-
 .../com/google/protobuf/ByteBufferWriter.java   |   50 +-
 .../shaded/com/google/protobuf/ByteString.java  |   18 +-
 .../shaded/com/google/protobuf/BytesValue.java  |   10 +-
 .../google/protobuf/BytesValueOrBuilder.java    |    2 +-
 .../com/google/protobuf/CodedInputStream.java   |    7 +-
 .../com/google/protobuf/DescriptorProtos.java   |  701 +++++++++--
 .../shaded/com/google/protobuf/Descriptors.java |    2 +-
 .../shaded/com/google/protobuf/DoubleValue.java |   10 +-
 .../google/protobuf/DoubleValueOrBuilder.java   |    2 +-
 .../shaded/com/google/protobuf/Duration.java    |   18 +-
 .../com/google/protobuf/DurationOrBuilder.java  |    4 +-
 .../com/google/protobuf/DynamicMessage.java     |    2 +-
 .../hbase/shaded/com/google/protobuf/Empty.java |    2 +-
 .../hbase/shaded/com/google/protobuf/Enum.java  |   54 +-
 .../com/google/protobuf/EnumOrBuilder.java      |   14 +-
 .../shaded/com/google/protobuf/EnumValue.java   |   24 +-
 .../com/google/protobuf/EnumValueOrBuilder.java |    6 +-
 .../com/google/protobuf/ExtensionRegistry.java  |    1 -
 .../google/protobuf/ExtensionRegistryLite.java  |    7 +-
 .../hbase/shaded/com/google/protobuf/Field.java |  110 +-
 .../shaded/com/google/protobuf/FieldMask.java   |    2 +-
 .../com/google/protobuf/FieldMaskProto.java     |    7 +-
 .../com/google/protobuf/FieldOrBuilder.java     |   30 +-
 .../shaded/com/google/protobuf/FieldSet.java    |    1 +
 .../shaded/com/google/protobuf/FloatValue.java  |   10 +-
 .../google/protobuf/FloatValueOrBuilder.java    |    2 +-
 .../google/protobuf/GeneratedMessageLite.java   |  163 ++-
 .../com/google/protobuf/GeneratedMessageV3.java |   26 +-
 .../shaded/com/google/protobuf/Int32Value.java  |   10 +-
 .../google/protobuf/Int32ValueOrBuilder.java    |    2 +-
 .../shaded/com/google/protobuf/Int64Value.java  |   10 +-
 .../google/protobuf/Int64ValueOrBuilder.java    |    2 +-
 .../com/google/protobuf/LazyFieldLite.java      |   25 +-
 .../shaded/com/google/protobuf/ListValue.java   |    2 +-
 .../shaded/com/google/protobuf/MapEntry.java    |    2 +-
 .../com/google/protobuf/MapFieldLite.java       |    4 +-
 .../google/protobuf/MessageLiteToString.java    |    4 +-
 .../shaded/com/google/protobuf/Method.java      |   74 +-
 .../com/google/protobuf/MethodOrBuilder.java    |   20 +-
 .../hbase/shaded/com/google/protobuf/Mixin.java |   30 +-
 .../com/google/protobuf/MixinOrBuilder.java     |    8 +-
 .../shaded/com/google/protobuf/NullValue.java   |    3 +-
 .../shaded/com/google/protobuf/Option.java      |  135 +-
 .../com/google/protobuf/OptionOrBuilder.java    |   35 +-
 .../com/google/protobuf/SmallSortedMap.java     |   21 +-
 .../com/google/protobuf/SourceContext.java      |   16 +-
 .../google/protobuf/SourceContextOrBuilder.java |    4 +-
 .../com/google/protobuf/SourceContextProto.java |    8 +-
 .../shaded/com/google/protobuf/StringValue.java |   16 +-
 .../google/protobuf/StringValueOrBuilder.java   |    4 +-
 .../shaded/com/google/protobuf/Struct.java      |   14 +-
 .../shaded/com/google/protobuf/Syntax.java      |    3 +-
 .../shaded/com/google/protobuf/Timestamp.java   |   18 +-
 .../com/google/protobuf/TimestampOrBuilder.java |    4 +-
 .../hbase/shaded/com/google/protobuf/Type.java  |   54 +-
 .../com/google/protobuf/TypeOrBuilder.java      |   14 +-
 .../shaded/com/google/protobuf/TypeProto.java   |    7 +-
 .../shaded/com/google/protobuf/UInt32Value.java |   10 +-
 .../google/protobuf/UInt32ValueOrBuilder.java   |    2 +-
 .../shaded/com/google/protobuf/UInt64Value.java |   10 +-
 .../google/protobuf/UInt64ValueOrBuilder.java   |    2 +-
 .../com/google/protobuf/UnknownFieldSet.java    |   35 +-
 .../google/protobuf/UnknownFieldSetLite.java    |   62 +-
 .../hbase/shaded/com/google/protobuf/Value.java |   86 +-
 .../com/google/protobuf/ValueOrBuilder.java     |   20 +-
 .../google/protobuf/compiler/PluginProtos.java  | 1178 +++++++++++++++++-
 .../protobuf/generated/TestProcedureProtos.java |    2 +-
 .../ipc/protobuf/generated/TestProtos.java      |   12 +-
 .../shaded/protobuf/generated/AdminProtos.java  |   76 +-
 .../shaded/protobuf/generated/BackupProtos.java |   10 +-
 .../shaded/protobuf/generated/CellProtos.java   |    4 +-
 .../shaded/protobuf/generated/ClientProtos.java |   72 +-
 .../protobuf/generated/ClusterIdProtos.java     |    2 +-
 .../protobuf/generated/ClusterStatusProtos.java |   20 +-
 .../protobuf/generated/ComparatorProtos.java    |   18 +-
 .../protobuf/generated/EncryptionProtos.java    |    2 +-
 .../protobuf/generated/ErrorHandlingProtos.java |    6 +-
 .../shaded/protobuf/generated/FSProtos.java     |    4 +-
 .../shaded/protobuf/generated/FilterProtos.java |   60 +-
 .../shaded/protobuf/generated/HBaseProtos.java  |   50 +-
 .../shaded/protobuf/generated/HFileProtos.java  |    4 +-
 .../protobuf/generated/LoadBalancerProtos.java  |    2 +-
 .../protobuf/generated/LockServiceProtos.java   |   10 +-
 .../protobuf/generated/MapReduceProtos.java     |    4 +-
 .../generated/MasterProcedureProtos.java        |   36 +-
 .../shaded/protobuf/generated/MasterProtos.java |  246 ++--
 .../protobuf/generated/ProcedureProtos.java     |   16 +-
 .../shaded/protobuf/generated/QuotaProtos.java  |   10 +-
 .../shaded/protobuf/generated/RPCProtos.java    |   16 +-
 .../generated/RegionNormalizerProtos.java       |    2 +-
 .../generated/RegionServerStatusProtos.java     |   26 +-
 .../protobuf/generated/ReplicationProtos.java   |   38 +-
 .../protobuf/generated/SnapshotProtos.java      |   10 +-
 .../protobuf/generated/TracingProtos.java       |    2 +-
 .../shaded/protobuf/generated/WALProtos.java    |   20 +-
 .../protobuf/generated/ZooKeeperProtos.java     |   12 +-
 105 files changed, 3043 insertions(+), 1128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java
index c294f10..e5b87f0 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java
@@ -48,7 +48,6 @@ public abstract class AbstractMessageLite<
     BuilderType extends AbstractMessageLite.Builder<MessageType, BuilderType>>
         implements MessageLite {
   protected int memoizedHashCode = 0;
-
   @Override
   public ByteString toByteString() {
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractParser.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractParser.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractParser.java
index 42924cd..5fe6a22 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractParser.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractParser.java
@@ -232,7 +232,7 @@ public abstract class AbstractParser<MessageType extends MessageLite>
       }
       size = CodedInputStream.readRawVarint32(firstByte, input);
     } catch (IOException e) {
-      throw new InvalidProtocolBufferException(e.getMessage());
+      throw new InvalidProtocolBufferException(e);
     }
     InputStream limitedInput = new LimitedInputStream(input, size);
     return parsePartialFrom(limitedInput, extensionRegistry);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Any.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Any.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Any.java
index eba424a..bf8e852 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Any.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Any.java
@@ -160,7 +160,7 @@ public  final class Any extends
   }
 
   /**
-   * Packs a message uisng the given type URL prefix. The type URL will
+   * Packs a message using the given type URL prefix. The type URL will
    * be constructed by concatenating the message type's full name to the
    * prefix with an optional "/" separator if the prefix doesn't end
    * with "/" already.
@@ -225,7 +225,7 @@ public  final class Any extends
    * used with implementation specific semantics.
    * </pre>
    *
-   * <code>optional string type_url = 1;</code>
+   * <code>string type_url = 1;</code>
    */
   public java.lang.String getTypeUrl() {
     java.lang.Object ref = typeUrl_;
@@ -261,7 +261,7 @@ public  final class Any extends
    * used with implementation specific semantics.
    * </pre>
    *
-   * <code>optional string type_url = 1;</code>
+   * <code>string type_url = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getTypeUrlBytes() {
@@ -284,7 +284,7 @@ public  final class Any extends
    * Must be a valid serialized protocol buffer of the above specified type.
    * </pre>
    *
-   * <code>optional bytes value = 2;</code>
+   * <code>bytes value = 2;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
     return value_;
@@ -351,7 +351,7 @@ public  final class Any extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + TYPE_URL_FIELD_NUMBER;
     hash = (53 * hash) + getTypeUrl().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
@@ -657,7 +657,7 @@ public  final class Any extends
      * used with implementation specific semantics.
      * </pre>
      *
-     * <code>optional string type_url = 1;</code>
+     * <code>string type_url = 1;</code>
      */
     public java.lang.String getTypeUrl() {
       java.lang.Object ref = typeUrl_;
@@ -693,7 +693,7 @@ public  final class Any extends
      * used with implementation specific semantics.
      * </pre>
      *
-     * <code>optional string type_url = 1;</code>
+     * <code>string type_url = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getTypeUrlBytes() {
@@ -730,7 +730,7 @@ public  final class Any extends
      * used with implementation specific semantics.
      * </pre>
      *
-     * <code>optional string type_url = 1;</code>
+     * <code>string type_url = 1;</code>
      */
     public Builder setTypeUrl(
         java.lang.String value) {
@@ -764,7 +764,7 @@ public  final class Any extends
      * used with implementation specific semantics.
      * </pre>
      *
-     * <code>optional string type_url = 1;</code>
+     * <code>string type_url = 1;</code>
      */
     public Builder clearTypeUrl() {
       
@@ -794,7 +794,7 @@ public  final class Any extends
      * used with implementation specific semantics.
      * </pre>
      *
-     * <code>optional string type_url = 1;</code>
+     * <code>string type_url = 1;</code>
      */
     public Builder setTypeUrlBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -814,7 +814,7 @@ public  final class Any extends
      * Must be a valid serialized protocol buffer of the above specified type.
      * </pre>
      *
-     * <code>optional bytes value = 2;</code>
+     * <code>bytes value = 2;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
       return value_;
@@ -824,7 +824,7 @@ public  final class Any extends
      * Must be a valid serialized protocol buffer of the above specified type.
      * </pre>
      *
-     * <code>optional bytes value = 2;</code>
+     * <code>bytes value = 2;</code>
      */
     public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
       if (value == null) {
@@ -840,7 +840,7 @@ public  final class Any extends
      * Must be a valid serialized protocol buffer of the above specified type.
      * </pre>
      *
-     * <code>optional bytes value = 2;</code>
+     * <code>bytes value = 2;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AnyOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AnyOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AnyOrBuilder.java
index f22f47a..80981c8 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AnyOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AnyOrBuilder.java
@@ -29,7 +29,7 @@ public interface AnyOrBuilder extends
    * used with implementation specific semantics.
    * </pre>
    *
-   * <code>optional string type_url = 1;</code>
+   * <code>string type_url = 1;</code>
    */
   java.lang.String getTypeUrl();
   /**
@@ -54,7 +54,7 @@ public interface AnyOrBuilder extends
    * used with implementation specific semantics.
    * </pre>
    *
-   * <code>optional string type_url = 1;</code>
+   * <code>string type_url = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getTypeUrlBytes();
@@ -64,7 +64,7 @@ public interface AnyOrBuilder extends
    * Must be a valid serialized protocol buffer of the above specified type.
    * </pre>
    *
-   * <code>optional bytes value = 2;</code>
+   * <code>bytes value = 2;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Api.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Api.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Api.java
index ea15b40..3265158 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Api.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/Api.java
@@ -151,7 +151,7 @@ public  final class Api extends
    * followed by the api's simple name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public java.lang.String getName() {
     java.lang.Object ref = name_;
@@ -171,7 +171,7 @@ public  final class Api extends
    * followed by the api's simple name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes() {
@@ -321,7 +321,7 @@ public  final class Api extends
    * experimental, none-GA apis.
    * </pre>
    *
-   * <code>optional string version = 4;</code>
+   * <code>string version = 4;</code>
    */
   public java.lang.String getVersion() {
     java.lang.Object ref = version_;
@@ -357,7 +357,7 @@ public  final class Api extends
    * experimental, none-GA apis.
    * </pre>
    *
-   * <code>optional string version = 4;</code>
+   * <code>string version = 4;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getVersionBytes() {
@@ -381,7 +381,7 @@ public  final class Api extends
    * message.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   public boolean hasSourceContext() {
     return sourceContext_ != null;
@@ -392,7 +392,7 @@ public  final class Api extends
    * message.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext() {
     return sourceContext_ == null ? org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.getDefaultInstance() : sourceContext_;
@@ -403,7 +403,7 @@ public  final class Api extends
    * message.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder() {
     return getSourceContext();
@@ -471,7 +471,7 @@ public  final class Api extends
    * The source syntax of the service.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   public int getSyntaxValue() {
     return syntax_;
@@ -481,7 +481,7 @@ public  final class Api extends
    * The source syntax of the service.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -595,7 +595,7 @@ public  final class Api extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + NAME_FIELD_NUMBER;
     hash = (53 * hash) + getName().hashCode();
     if (getMethodsCount() > 0) {
@@ -1000,7 +1000,7 @@ public  final class Api extends
      * followed by the api's simple name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public java.lang.String getName() {
       java.lang.Object ref = name_;
@@ -1020,7 +1020,7 @@ public  final class Api extends
      * followed by the api's simple name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getNameBytes() {
@@ -1041,7 +1041,7 @@ public  final class Api extends
      * followed by the api's simple name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setName(
         java.lang.String value) {
@@ -1059,7 +1059,7 @@ public  final class Api extends
      * followed by the api's simple name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder clearName() {
       
@@ -1073,7 +1073,7 @@ public  final class Api extends
      * followed by the api's simple name.
      * </pre>
      *
-     * <code>optional string name = 1;</code>
+     * <code>string name = 1;</code>
      */
     public Builder setNameBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -1734,7 +1734,7 @@ public  final class Api extends
      * experimental, none-GA apis.
      * </pre>
      *
-     * <code>optional string version = 4;</code>
+     * <code>string version = 4;</code>
      */
     public java.lang.String getVersion() {
       java.lang.Object ref = version_;
@@ -1770,7 +1770,7 @@ public  final class Api extends
      * experimental, none-GA apis.
      * </pre>
      *
-     * <code>optional string version = 4;</code>
+     * <code>string version = 4;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
         getVersionBytes() {
@@ -1807,7 +1807,7 @@ public  final class Api extends
      * experimental, none-GA apis.
      * </pre>
      *
-     * <code>optional string version = 4;</code>
+     * <code>string version = 4;</code>
      */
     public Builder setVersion(
         java.lang.String value) {
@@ -1841,7 +1841,7 @@ public  final class Api extends
      * experimental, none-GA apis.
      * </pre>
      *
-     * <code>optional string version = 4;</code>
+     * <code>string version = 4;</code>
      */
     public Builder clearVersion() {
       
@@ -1871,7 +1871,7 @@ public  final class Api extends
      * experimental, none-GA apis.
      * </pre>
      *
-     * <code>optional string version = 4;</code>
+     * <code>string version = 4;</code>
      */
     public Builder setVersionBytes(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
@@ -1894,7 +1894,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public boolean hasSourceContext() {
       return sourceContextBuilder_ != null || sourceContext_ != null;
@@ -1905,7 +1905,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext() {
       if (sourceContextBuilder_ == null) {
@@ -1920,7 +1920,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder setSourceContext(org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext value) {
       if (sourceContextBuilder_ == null) {
@@ -1941,7 +1941,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder setSourceContext(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder builderForValue) {
@@ -1960,7 +1960,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder mergeSourceContext(org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext value) {
       if (sourceContextBuilder_ == null) {
@@ -1983,7 +1983,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public Builder clearSourceContext() {
       if (sourceContextBuilder_ == null) {
@@ -2002,7 +2002,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder getSourceContextBuilder() {
       
@@ -2015,7 +2015,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder() {
       if (sourceContextBuilder_ != null) {
@@ -2031,7 +2031,7 @@ public  final class Api extends
      * message.
      * </pre>
      *
-     * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+     * <code>.google.protobuf.SourceContext source_context = 5;</code>
      */
     private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
         org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext, org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext.Builder, org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder> 
@@ -2365,7 +2365,7 @@ public  final class Api extends
      * The source syntax of the service.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public int getSyntaxValue() {
       return syntax_;
@@ -2375,7 +2375,7 @@ public  final class Api extends
      * The source syntax of the service.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public Builder setSyntaxValue(int value) {
       syntax_ = value;
@@ -2387,7 +2387,7 @@ public  final class Api extends
      * The source syntax of the service.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax() {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax result = org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax.valueOf(syntax_);
@@ -2398,7 +2398,7 @@ public  final class Api extends
      * The source syntax of the service.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public Builder setSyntax(org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax value) {
       if (value == null) {
@@ -2414,7 +2414,7 @@ public  final class Api extends
      * The source syntax of the service.
      * </pre>
      *
-     * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+     * <code>.google.protobuf.Syntax syntax = 7;</code>
      */
     public Builder clearSyntax() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiOrBuilder.java
index 5aea779..b8b7030 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiOrBuilder.java
@@ -13,7 +13,7 @@ public interface ApiOrBuilder extends
    * followed by the api's simple name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   java.lang.String getName();
   /**
@@ -22,7 +22,7 @@ public interface ApiOrBuilder extends
    * followed by the api's simple name.
    * </pre>
    *
-   * <code>optional string name = 1;</code>
+   * <code>string name = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getNameBytes();
@@ -137,7 +137,7 @@ public interface ApiOrBuilder extends
    * experimental, none-GA apis.
    * </pre>
    *
-   * <code>optional string version = 4;</code>
+   * <code>string version = 4;</code>
    */
   java.lang.String getVersion();
   /**
@@ -162,7 +162,7 @@ public interface ApiOrBuilder extends
    * experimental, none-GA apis.
    * </pre>
    *
-   * <code>optional string version = 4;</code>
+   * <code>string version = 4;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
       getVersionBytes();
@@ -173,7 +173,7 @@ public interface ApiOrBuilder extends
    * message.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   boolean hasSourceContext();
   /**
@@ -182,7 +182,7 @@ public interface ApiOrBuilder extends
    * message.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContext getSourceContext();
   /**
@@ -191,7 +191,7 @@ public interface ApiOrBuilder extends
    * message.
    * </pre>
    *
-   * <code>optional .google.protobuf.SourceContext source_context = 5;</code>
+   * <code>.google.protobuf.SourceContext source_context = 5;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.SourceContextOrBuilder getSourceContextOrBuilder();
 
@@ -244,7 +244,7 @@ public interface ApiOrBuilder extends
    * The source syntax of the service.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   int getSyntaxValue();
   /**
@@ -252,7 +252,7 @@ public interface ApiOrBuilder extends
    * The source syntax of the service.
    * </pre>
    *
-   * <code>optional .google.protobuf.Syntax syntax = 7;</code>
+   * <code>.google.protobuf.Syntax syntax = 7;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.Syntax getSyntax();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiProto.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiProto.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiProto.java
index 4eb312a..817e07d 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiProto.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ApiProto.java
@@ -53,9 +53,10 @@ public final class ApiProto {
       "esponse_streaming\030\005 \001(\010\022(\n\007options\030\006 \003(\013" +
       "2\027.google.protobuf.Option\022\'\n\006syntax\030\007 \001(" +
       "\0162\027.google.protobuf.Syntax\"#\n\005Mixin\022\014\n\004n" +
-      "ame\030\001 \001(\t\022\014\n\004root\030\002 \001(\tBH\n\023com.google.pr" +
-      "otobufB\010ApiProtoP\001\242\002\003GPB\252\002\036Google.Protob" +
-      "uf.WellKnownTypesb\006proto3"
+      "ame\030\001 \001(\t\022\014\n\004root\030\002 \001(\tBu\n\023com.google.pr" +
+      "otobufB\010ApiProtoP\001Z+google.golang.org/ge" +
+      "nproto/protobuf/api;api\242\002\003GPB\252\002\036Google.P" +
+      "rotobuf.WellKnownTypesb\006proto3"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValue.java
index 6e9895f..a2dbe4a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValue.java
@@ -83,7 +83,7 @@ public  final class BoolValue extends
    * The bool value.
    * </pre>
    *
-   * <code>optional bool value = 1;</code>
+   * <code>bool value = 1;</code>
    */
   public boolean getValue() {
     return value_;
@@ -142,7 +142,7 @@ public  final class BoolValue extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
         getValue());
@@ -370,7 +370,7 @@ public  final class BoolValue extends
      * The bool value.
      * </pre>
      *
-     * <code>optional bool value = 1;</code>
+     * <code>bool value = 1;</code>
      */
     public boolean getValue() {
       return value_;
@@ -380,7 +380,7 @@ public  final class BoolValue extends
      * The bool value.
      * </pre>
      *
-     * <code>optional bool value = 1;</code>
+     * <code>bool value = 1;</code>
      */
     public Builder setValue(boolean value) {
       
@@ -393,7 +393,7 @@ public  final class BoolValue extends
      * The bool value.
      * </pre>
      *
-     * <code>optional bool value = 1;</code>
+     * <code>bool value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValueOrBuilder.java
index e1d4e75..c1d3b5f 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BoolValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface BoolValueOrBuilder extends
    * The bool value.
    * </pre>
    *
-   * <code>optional bool value = 1;</code>
+   * <code>bool value = 1;</code>
    */
   boolean getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteBufferWriter.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteBufferWriter.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteBufferWriter.java
index 906c216..553243a 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteBufferWriter.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteBufferWriter.java
@@ -33,11 +33,12 @@ package org.apache.hadoop.hbase.shaded.com.google.protobuf;
 import static java.lang.Math.max;
 import static java.lang.Math.min;
 
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.lang.ref.SoftReference;
+import java.lang.reflect.Field;
 import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
 
 /**
  * Utility class to provide efficient writing of {@link ByteBuffer}s to {@link OutputStream}s.
@@ -75,6 +76,12 @@ final class ByteBufferWriter {
       new ThreadLocal<SoftReference<byte[]>>();
 
   /**
+   * This is a hack for GAE, where {@code FileOutputStream} is unavailable.
+   */
+  private static final Class<?> FILE_OUTPUT_STREAM_CLASS = safeGetClass("java.io.FileOutputStream");
+  private static final long CHANNEL_FIELD_OFFSET = getChannelFieldOffset(FILE_OUTPUT_STREAM_CLASS);
+
+  /**
    * For testing purposes only. Clears the cached buffer to force a new allocation on the next
    * invocation.
    */
@@ -93,10 +100,7 @@ final class ByteBufferWriter {
         // Optimized write for array-backed buffers.
         // Note that we're taking the risk that a malicious OutputStream could modify the array.
         output.write(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
-      } else if (output instanceof FileOutputStream) {
-        // Use a channel to write out the ByteBuffer. This will automatically empty the buffer.
-        ((FileOutputStream) output).getChannel().write(buffer);
-      } else {
+      } else if (!writeToChannel(buffer, output)){
         // Read all of the data from the buffer to an array.
         // TODO(nathanmittler): Consider performance improvements for other "known" stream types.
         final byte[] array = getOrCreateBuffer(buffer.remaining());
@@ -142,4 +146,40 @@ final class ByteBufferWriter {
   private static void setBuffer(byte[] value) {
     BUFFER.set(new SoftReference<byte[]>(value));
   }
+
+  private static boolean writeToChannel(ByteBuffer buffer, OutputStream output) throws IOException {
+    if (CHANNEL_FIELD_OFFSET >= 0 && FILE_OUTPUT_STREAM_CLASS.isInstance(output)) {
+      // Use a channel to write out the ByteBuffer. This will automatically empty the buffer.
+      WritableByteChannel channel = null;
+      try {
+        channel = (WritableByteChannel) UnsafeUtil.getObject(output, CHANNEL_FIELD_OFFSET);
+      } catch (ClassCastException e) {
+        // Absorb.
+      }
+      if (channel != null) {
+        channel.write(buffer);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static Class<?> safeGetClass(String className) {
+    try {
+      return Class.forName(className);
+    } catch (ClassNotFoundException e) {
+      return null;
+    }
+  }
+  private static long getChannelFieldOffset(Class<?> clazz) {
+    try {
+      if (clazz != null && UnsafeUtil.hasUnsafeArrayOperations()) {
+        Field field = clazz.getDeclaredField("channel");
+        return UnsafeUtil.objectFieldOffset(field);
+      }
+    } catch (Throwable e) {
+      // Absorb
+    }
+    return -1;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteString.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteString.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteString.java
index ebfa7fa..f673ee3 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteString.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/ByteString.java
@@ -51,14 +51,12 @@ import java.util.List;
 import java.util.NoSuchElementException;
 
 /**
- * Immutable sequence of bytes.  Substring is supported by sharing the reference
- * to the immutable underlying bytes.  Concatenation is likewise supported
- * without copying (long strings) by building a tree of pieces in
- * {@link RopeByteString}.
- * <p>
- * Like {@link String}, the contents of a {@link ByteString} can never be
- * observed to change, not even in the presence of a data race or incorrect
- * API usage in the client code.
+ * Immutable sequence of bytes. Substring is supported by sharing the reference to the immutable
+ * underlying bytes. Concatenation is likewise supported without copying (long strings) by building
+ * a tree of pieces in {@link RopeByteString}.
+ *
+ * <p>Like {@link String}, the contents of a {@link ByteString} can never be observed to change, not
+ * even in the presence of a data race or incorrect API usage in the client code.
  *
  * @author crazybob@google.com Bob Lee
  * @author kenton@google.com Kenton Varda
@@ -572,7 +570,9 @@ public abstract class ByteString implements Iterable<Byte>, Serializable {
   // Create a balanced concatenation of the next "length" elements from the
   // iterable.
   private static ByteString balancedConcat(Iterator<ByteString> iterator, int length) {
-    assert length >= 1;
+    if (length < 1) {
+      throw new IllegalArgumentException(String.format("length (%s) must be >= 1", length));
+    }
     ByteString result;
     if (length == 1) {
       result = iterator.next();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValue.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValue.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValue.java
index 4349b0d..ecfbcc1 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValue.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValue.java
@@ -83,7 +83,7 @@ public  final class BytesValue extends
    * The bytes value.
    * </pre>
    *
-   * <code>optional bytes value = 1;</code>
+   * <code>bytes value = 1;</code>
    */
   public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
     return value_;
@@ -142,7 +142,7 @@ public  final class BytesValue extends
       return memoizedHashCode;
     }
     int hash = 41;
-    hash = (19 * hash) + getDescriptorForType().hashCode();
+    hash = (19 * hash) + getDescriptor().hashCode();
     hash = (37 * hash) + VALUE_FIELD_NUMBER;
     hash = (53 * hash) + getValue().hashCode();
     hash = (29 * hash) + unknownFields.hashCode();
@@ -369,7 +369,7 @@ public  final class BytesValue extends
      * The bytes value.
      * </pre>
      *
-     * <code>optional bytes value = 1;</code>
+     * <code>bytes value = 1;</code>
      */
     public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue() {
       return value_;
@@ -379,7 +379,7 @@ public  final class BytesValue extends
      * The bytes value.
      * </pre>
      *
-     * <code>optional bytes value = 1;</code>
+     * <code>bytes value = 1;</code>
      */
     public Builder setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
       if (value == null) {
@@ -395,7 +395,7 @@ public  final class BytesValue extends
      * The bytes value.
      * </pre>
      *
-     * <code>optional bytes value = 1;</code>
+     * <code>bytes value = 1;</code>
      */
     public Builder clearValue() {
       

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValueOrBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValueOrBuilder.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValueOrBuilder.java
index 9511406..3f90289 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValueOrBuilder.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/BytesValueOrBuilder.java
@@ -12,7 +12,7 @@ public interface BytesValueOrBuilder extends
    * The bytes value.
    * </pre>
    *
-   * <code>optional bytes value = 1;</code>
+   * <code>bytes value = 1;</code>
    */
   org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getValue();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e916b79d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/CodedInputStream.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/CodedInputStream.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/CodedInputStream.java
index 0bff626..23cc1a4 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/CodedInputStream.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/CodedInputStream.java
@@ -60,7 +60,8 @@ import java.util.List;
 public abstract class CodedInputStream {
   private static final int DEFAULT_BUFFER_SIZE = 4096;
   private static final int DEFAULT_RECURSION_LIMIT = 100;
-  private static final int DEFAULT_SIZE_LIMIT = 64 << 20; // 64MB
+  // Integer.MAX_VALUE == 0x7FFFFFF == INT_MAX from limits.h
+  private static final int DEFAULT_SIZE_LIMIT = Integer.MAX_VALUE;
 
   /** Visible for subclasses. See setRecursionLimit() */
   int recursionDepth;
@@ -2767,9 +2768,9 @@ public abstract class CodedInputStream {
         throw InvalidProtocolBufferException.negativeSize();
       }
 
-      // Verify that the message size so far has not exceeded sizeLimit.
+      // Integer-overflow-conscious check that the message size so far has not exceeded sizeLimit.
       int currentMessageSize = totalBytesRetired + pos + size;
-      if (currentMessageSize > sizeLimit) {
+      if (currentMessageSize - sizeLimit > 0) {
         throw InvalidProtocolBufferException.sizeLimitExceeded();
       }
 


[17/50] [abbrv] hbase git commit: HBASE-17785 RSGroupBasedLoadBalancer fails to assign new table regions when cloning snapshot

Posted by sy...@apache.org.
HBASE-17785 RSGroupBasedLoadBalancer fails to assign new table regions when cloning snapshot


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

Branch: refs/heads/hbase-12439
Commit: 029fa297129f7ced276d19c4877d19bf32dcfde0
Parents: cbcbcf4
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Apr 5 16:25:56 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Apr 5 16:25:56 2017 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     | 28 ++++++++++++++------
 .../hadoop/hbase/rsgroup/TestRSGroups.java      | 19 +++++++++++++
 2 files changed, 39 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/029fa297/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 83389e4..14907ba 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveTablesR
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RSGroupAdminService;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 @InterfaceAudience.Private
 public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService {
@@ -267,14 +268,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
     }
   }
 
-  /////////////////////////////////////////////////////////////////////////////
-  // MasterObserver overrides
-  /////////////////////////////////////////////////////////////////////////////
-
-  // Assign table to default RSGroup.
-  @Override
-  public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+  void assignTableToGroup(HTableDescriptor desc) throws IOException {
     String groupName =
         master.getClusterSchema().getNamespace(desc.getTableName().getNamespaceAsString())
                 .getConfigurationValue(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP);
@@ -292,6 +286,17 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
     }
   }
 
+  /////////////////////////////////////////////////////////////////////////////
+  // MasterObserver overrides
+  /////////////////////////////////////////////////////////////////////////////
+
+  // Assign table to default RSGroup.
+  @Override
+  public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+    assignTableToGroup(desc);
+  }
+
   // Remove table from its RSGroup.
   @Override
   public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -322,5 +327,12 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
                                  NamespaceDescriptor ns) throws IOException {
     preCreateNamespace(ctx, ns);
   }
+
+  @Override
+  public void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      SnapshotDescription snapshot, HTableDescriptor desc) throws IOException {
+    assignTableToGroup(desc);
+  }
+
   /////////////////////////////////////////////////////////////////////////////
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/029fa297/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index 3886684..d6bd43b 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
@@ -77,6 +78,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     TEST_UTIL.getConfiguration().set(
         ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
         ""+NUM_SLAVES_BASE);
+    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
 
     admin = TEST_UTIL.getAdmin();
     cluster = TEST_UTIL.getHBaseCluster();
@@ -270,4 +272,21 @@ public class TestRSGroups extends TestRSGroupsBase {
       }
     });
   }
+
+  @Test
+  public void testCloneSnapshot() throws Exception {
+    byte[] FAMILY = Bytes.toBytes("test");
+    String snapshotName = tableName.getNameAsString() + "_snap";
+    TableName clonedTableName = TableName.valueOf(tableName.getNameAsString() + "_clone");
+
+    // create base table
+    TEST_UTIL.createTable(tableName, FAMILY);
+
+    // create snapshot
+    admin.snapshot(snapshotName, tableName);
+
+    // clone
+    admin.cloneSnapshot(snapshotName, clonedTableName);
+  }
+
 }


[13/50] [abbrv] hbase git commit: HBASE-17857 Remove IS annotations from IA.Public classes

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
index 7aa807c..ed95a7d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 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.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@@ -40,7 +39,6 @@ import com.google.common.base.Preconditions;
  * Use this filter to include the stop row, eg: [A,Z].
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class InclusiveStopFilter extends FilterBase {
   private byte [] stopRowKey;
   private boolean done = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java
index 8eba03c..6410ab4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Used to indicate a filter incompatibility
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class IncompatibleFilterException extends RuntimeException {
   private static final long serialVersionUID = 3236763276623198231L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
index 81aae0b..0406058 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
@@ -19,13 +19,11 @@
 package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Used to indicate an invalid RowFilter.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class InvalidRowFilterException extends RuntimeException {
   private static final long serialVersionUID = 2667894046345657865L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index adbf304..b082941 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 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.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -43,7 +42,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * the values.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class KeyOnlyFilter extends FilterBase {
 
   boolean lenAsVal;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
index 8bcc7b2..429b498 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
@@ -23,7 +23,6 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
@@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * A long comparator which numerical compares against the specified byte array
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class LongComparator extends ByteArrayComparable {
   private long longValue;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
index 77fbaf4..d398349 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@@ -48,7 +47,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * specified list and perform fast-forwarding during scan. Thus, the scan will be quite efficient.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class MultiRowRangeFilter extends FilterBase {
 
   private List<RowRange> rangeList;
@@ -413,7 +411,6 @@ public class MultiRowRangeFilter extends FilterBase {
   }
 
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public static class RowRange implements Comparable<RowRange> {
     private byte[] startRow;
     private boolean startRowInclusive = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
index 12d9ac7..6bcb561 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
@@ -25,7 +25,6 @@ import java.util.TreeSet;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * columns like 'and', 'anti' but not keys with columns like 'ball', 'act'.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class MultipleColumnPrefixFilter extends FilterBase {
   protected byte [] hint = null;
   protected TreeSet<byte []> sortedPrefixes = createTreeSet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
index 0d60e2e..a72afca 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
@@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.filter;
 import java.nio.ByteBuffer;
 
 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.shaded.protobuf.generated.ComparatorProtos;
 
@@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class NullComparator extends ByteArrayComparable {
 
   public NullComparator() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
index 2b91b7a..894e7b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.generated.FilterProtos;
 
@@ -41,7 +40,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * locally.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class PageFilter extends FilterBase {
   private long pageSize = Long.MAX_VALUE;
   private int rowsAccepted = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java
index 3a20772..785f3f3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java
@@ -21,14 +21,12 @@ package org.apache.hadoop.hbase.filter;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * ParseConstants holds a bunch of constants related to parsing Filter Strings
  * Used by {@link ParseFilter}
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public final class ParseConstants {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
index 0823785..21cdd9c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
@@ -33,7 +33,6 @@ import java.util.Stack;
 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.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -47,7 +46,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * Filter Language can be found at: https://issues.apache.org/jira/browse/HBASE-4176
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ParseFilter {
   private static final Log LOG = LogFactory.getLog(ParseFilter.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
index e3cefe5..33b3ead 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
  * Pass results that have same row prefix.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class PrefixFilter extends FilterBase {
   protected byte [] prefix = null;
   protected boolean passedPrefix = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
index cc240f8..72a50fb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * directly rather than a filter.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class QualifierFilter extends CompareFilter {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
index 177ed4d..48413ac 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
@@ -23,7 +23,6 @@ import java.util.Random;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.generated.FilterProtos;
 
@@ -34,7 +33,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * 
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RandomRowFilter extends FilterBase {
   protected static final Random random = new Random();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
index 3f05901..2f5a342 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
@@ -26,7 +26,6 @@ import java.util.regex.Pattern;
 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.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -71,7 +70,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * @see java.util.regex.Pattern
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RegexStringComparator extends ByteArrayComparable {
 
   private static final Log LOG = LogFactory.getLog(RegexStringComparator.class);
@@ -80,7 +78,6 @@ public class RegexStringComparator extends ByteArrayComparable {
 
   /** Engine implementation type (default=JAVA) */
   @InterfaceAudience.Public
-  @InterfaceStability.Stable
   public enum EngineType {
     JAVA,
     JONI

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
index 4f91f8b..3f6136f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * and stop rows directly rather than a filter.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class RowFilter extends CompareFilter {
 
   private boolean filterOutRow = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
index 192fd97..6b155b0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
@@ -27,7 +27,6 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -41,7 +40,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * needed as input (besides for the filtering itself).
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
index 3bb80cb..0dbc0bb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 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.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
@@ -68,7 +67,6 @@ import com.google.common.base.Preconditions;
  * To filter based on the value of all scanned columns, use {@link ValueFilter}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class SingleColumnValueFilter extends FilterBase {
 
   protected byte [] columnFamily;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
index 5461011..1cdf206 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -51,7 +50,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * </p>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class SkipFilter extends FilterBase {
   private boolean filterRow = false;
   private Filter filter;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
index d30d057..157d97c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.filter;
 
 import java.util.Locale;
 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.shaded.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * </pre>
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class SubstringComparator extends ByteArrayComparable {
 
   private String substr;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
index 8c58f91..b1409e3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
@@ -25,7 +25,6 @@ import java.util.TreeSet;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.shaded.protobuf.generated.FilterProtos;
 
@@ -42,7 +41,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * or {@link org.apache.hadoop.hbase.client.Scan#setTimeStamp(long)}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class TimestampsFilter extends FilterBase {
 
   private final boolean canHint;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
index c0dacaf..4edb57b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * use {@link SingleColumnValueFilter}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ValueFilter extends CompareFilter {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
index 8738962..6de3676 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
  * returns true.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class WhileMatchFilter extends FilterBase {
   private boolean filterAllRemaining = false;
   private Filter filter;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BadAuthException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BadAuthException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BadAuthException.java
index 7eb96d5..010dbb9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BadAuthException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BadAuthException.java
@@ -18,10 +18,8 @@
 package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class BadAuthException extends FatalConnectionException {
   public BadAuthException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java
index a6777c0..d7f8c1e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallCancelledException.java
@@ -19,13 +19,11 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Client side call cancelled.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CallCancelledException extends HBaseIOException {
 
   private static final long serialVersionUID = 309775809470318208L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
index db8c34a..9a67aeb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Client-side call timeout
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CallTimeoutException extends HBaseIOException {
 
   public CallTimeoutException(final String msg) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallerDisconnectedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallerDisconnectedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallerDisconnectedException.java
index 6fd038f..c6dec2d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallerDisconnectedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallerDisconnectedException.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.ipc;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Exception indicating that the remote host making this IPC lost its
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * but is only used for logging on the server side, etc.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CallerDisconnectedException extends IOException {
   private static final long serialVersionUID = 1L;
   public CallerDisconnectedException(String msg) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java
index ffd27b3..d93d9f6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellScannerButNoCodecException.java
@@ -19,13 +19,11 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if a cellscanner but no codec to encode it with.
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CellScannerButNoCodecException extends HBaseIOException {
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
index 7ed1f7d..a9c10ce 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.ipc;
 import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.RpcChannel;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Base interface which provides clients with an RPC connection to
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface CoprocessorRpcChannel extends RpcChannel, BlockingRpcChannel {}
 // This Interface is part of our public, client-facing API!!!
-// This belongs in client package but it is exposed in our public API so we cannot relocate.
\ No newline at end of file
+// This belongs in client package but it is exposed in our public API so we cannot relocate.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java
index 12f6451..e50a82e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Indicates that we're trying to connect to a already known as dead server. We will want to
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FailedServerException extends HBaseIOException {
   public FailedServerException(String s) {
     super(s);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java
index 721148b..ac9fa97 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FallbackDisallowedException.java
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Indicate that the rpc server tells client to fallback to simple auth but client is disabled to do
  * so.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FallbackDisallowedException extends HBaseIOException {
 
   private static final long serialVersionUID = -6942845066279358253L;
@@ -35,4 +33,4 @@ public class FallbackDisallowedException extends HBaseIOException {
     super("Server asks us to fall back to SIMPLE auth, "
         + "but this client is configured to only allow secure connections.");
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java
index 86d3b89..74bd42f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FatalConnectionException.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when server finds fatal issue w/ connection setup: e.g. bad rpc version
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FatalConnectionException extends DoNotRetryIOException {
   public FatalConnectionException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
index a8af69c..fe039d3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClientConfigHelper.java
@@ -28,7 +28,6 @@ import java.util.Map;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Pair;
 
 /**
@@ -39,7 +38,6 @@ import org.apache.hadoop.hbase.util.Pair;
  * the whole process.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class NettyRpcClientConfigHelper {
 
   public static final String EVENT_LOOP_CONFIG = "hbase.rpc.client.event-loop.config";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
index 0e50943..eff5b7f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 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.util.DynamicClassLoader;
 import org.apache.hadoop.ipc.RemoteException;
@@ -37,7 +36,6 @@ import org.apache.hadoop.ipc.RemoteException;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
     value = "DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED", justification = "None. Address sometime.")
 public class RemoteWithExtrasException extends RemoteException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerNotRunningYetException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerNotRunningYetException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerNotRunningYetException.java
index 6d0b9de..15db1c0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerNotRunningYetException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerNotRunningYetException.java
@@ -22,11 +22,9 @@ package org.apache.hadoop.hbase.ipc;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ServerNotRunningYetException extends IOException {
   public ServerNotRunningYetException(String s) {
     super(s);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
index 0dd8e64..e2ed361 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
@@ -22,16 +22,14 @@ import java.net.InetSocketAddress;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Throw this in RPC call if there are too many pending requests for one region server
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ServerTooBusyException extends DoNotRetryIOException {
   public ServerTooBusyException(InetSocketAddress address, long count) {
     super("Busy Server! " + count + " concurrent RPCs against " + address);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java
index a224a12..63ce25b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java
@@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class StoppedRpcClientException extends HBaseIOException {
   public StoppedRpcClientException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCellCodecException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCellCodecException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCellCodecException.java
index 3208876..19914e1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCellCodecException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCellCodecException.java
@@ -18,10 +18,8 @@
 package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class UnsupportedCellCodecException extends FatalConnectionException {
   public UnsupportedCellCodecException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCompressionCodecException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCompressionCodecException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCompressionCodecException.java
index 7ca7dd5..271bd2b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCompressionCodecException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCompressionCodecException.java
@@ -18,10 +18,8 @@
 package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class UnsupportedCompressionCodecException extends FatalConnectionException {
   public UnsupportedCompressionCodecException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java
index 12e4a7a..5f2fa07 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/UnsupportedCryptoException.java
@@ -19,10 +19,8 @@
 package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class UnsupportedCryptoException extends FatalConnectionException {
   public UnsupportedCryptoException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/WrongVersionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/WrongVersionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/WrongVersionException.java
index 73bd10d..d63c867 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/WrongVersionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/WrongVersionException.java
@@ -18,10 +18,8 @@
 package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class WrongVersionException extends FatalConnectionException {
   public WrongVersionException() {
     super();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
index e0386b5..0ab75da 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaExceededException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase.quotas;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Generic quota exceeded exception
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class QuotaExceededException extends DoNotRetryIOException {
   public QuotaExceededException(String msg) {
     super(msg);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaFilter.java
index 309dd9c..b8a99a6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaFilter.java
@@ -21,14 +21,12 @@ import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Strings;
 
 /**
  * Filter to use to filter the QuotaRetriever results.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class QuotaFilter {
   private Set<QuotaType> types = new HashSet<>();
   private boolean hasFilters = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
index fecd2d1..cba6a24 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaRetriever.java
@@ -29,7 +29,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
@@ -43,7 +42,6 @@ import org.apache.hadoop.util.StringUtils;
  * Scanner to iterate over the quota settings.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
   private static final Log LOG = LogFactory.getLog(QuotaRetriever.class);
 
@@ -182,4 +180,4 @@ public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
     scanner.init(conf, scan);
     return scanner;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaScope.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaScope.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaScope.java
index 2e215b6..4a7d241 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaScope.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaScope.java
@@ -18,14 +18,12 @@
 package org.apache.hadoop.hbase.quotas;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Describe the Scope of the quota rules.
  * The quota can be enforced at the cluster level or at machine level.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum QuotaScope {
   /**
    * The specified throttling rules will be applied at the cluster level.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
index ac6a396..193d165 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
@@ -21,12 +21,10 @@ import java.util.concurrent.TimeUnit;
 
 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.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public abstract class QuotaSettings {
   private final String userName;
   private final String namespace;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
index 1a8b934..3622a32 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
@@ -23,14 +23,12 @@ import java.util.concurrent.TimeUnit;
 
 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.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class QuotaSettingsFactory {
   static class QuotaGlobalsSettingsBypass extends QuotaSettings {
     private final boolean bypassGlobals;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
index 40a8b66..1ec649f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaType.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.quotas;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Describe the Quota Type.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum QuotaType {
   THROTTLE,
   GLOBAL_BYPASS,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleType.java
index 9b456c2..724c880 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleType.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.quotas;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Describe the Throttle Type.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum ThrottleType {
   /** Throttling based on the number of requests per time-unit */
   REQUEST_NUMBER,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottlingException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottlingException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottlingException.java
index 293e9c6..ec665ae 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottlingException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottlingException.java
@@ -22,7 +22,6 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Describe the throttling result.
@@ -32,12 +31,10 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  * as result of this exception.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ThrottlingException extends QuotaExceededException {
   private static final long serialVersionUID = 1406576492085155743L;
 
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public enum Type {
     NumRequestsExceeded,
     RequestSizeExceeded,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/BloomType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/BloomType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/BloomType.java
index 50b8b15..073233b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/BloomType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/BloomType.java
@@ -20,10 +20,8 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public enum BloomType {
   /**
    * Bloomfilters disabled

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java
index d1fdae3..e4de9c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/LeaseException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Reports a problem with a lease
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class LeaseException extends DoNotRetryIOException {
 
   private static final long serialVersionUID = 8179703995292418650L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/NoSuchColumnFamilyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/NoSuchColumnFamilyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/NoSuchColumnFamilyException.java
index d3b1ec1..e2b7bba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/NoSuchColumnFamilyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/NoSuchColumnFamilyException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if request for nonexistent column family.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class NoSuchColumnFamilyException extends DoNotRetryIOException {
   private static final long serialVersionUID = -6569952730832331274L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAbortedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAbortedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAbortedException.java
index ddc2270..54d973b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAbortedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAbortedException.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown by the region server when it is aborting.
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionServerAbortedException extends RegionServerStoppedException {
   public RegionServerAbortedException(String s) {
     super(s);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerRunningException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerRunningException.java
index a4a9720..e8651db 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerRunningException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerRunningException.java
@@ -21,14 +21,12 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown if the region server log directory exists (which indicates another
  * region server is running at the same address)
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionServerRunningException extends IOException {
   private static final long serialVersionUID = 1L << 31 - 1L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerStoppedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerStoppedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerStoppedException.java
index 95f697e..99af432 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerStoppedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerStoppedException.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown by the region server when it is in shutting down state.
@@ -29,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class RegionServerStoppedException extends IOException {
   public RegionServerStoppedException(String s) {
     super(s);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/WrongRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/WrongRegionException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/WrongRegionException.java
index c2460d4..eb69e33 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/WrongRegionException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/WrongRegionException.java
@@ -21,13 +21,11 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when a request contains a key which is not part of this region
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class WrongRegionException extends IOException {
   private static final long serialVersionUID = 993179627856392526L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
index cc42819..c614a57 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
@@ -21,14 +21,12 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when we fail close of the write-ahead-log file.
  * Package private.  Only used inside this package.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FailedLogCloseException extends IOException {
   private static final long serialVersionUID = 1759152841462990925L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java
index 4c6ef45..ff79716 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java
@@ -19,14 +19,12 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Thrown when we fail close of the write-ahead-log file.
  * Package private.  Only used inside this package.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class FailedSyncBeforeLogCloseException extends FailedLogCloseException {
   private static final long serialVersionUID = 1759152841462990925L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationException.java
index 937e943..66781f1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationException.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.HBaseException;
 
 /**
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.exceptions.HBaseException;
  * store, loss of connection to a peer cluster or errors during deserialization of replication data.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Stable
 public class ReplicationException extends HBaseException {
 
   private static final long serialVersionUID = -8885598603988198062L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index f7cc2dd..badec0e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -27,14 +27,12 @@ import java.util.TreeMap;
 
 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.util.Bytes;
 
 /**
  * A configuration for the replication peer cluster.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ReplicationPeerConfig {
 
   private String clusterKey;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
index 577d13a..95c84c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
@@ -18,13 +18,11 @@
 package org.apache.hadoop.hbase.replication;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class ReplicationPeerDescription {
 
   private final String id;
@@ -56,4 +54,4 @@ public class ReplicationPeerDescription {
     builder.append(", config : " + config);
     return builder.toString();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
index 07b871d..cf57517 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AccessDeniedException.java
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.security;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 
 /**
  * Exception thrown by access-related methods.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class AccessDeniedException extends DoNotRetryIOException {
   private static final long serialVersionUID = 1913879564363001780L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
index 1c4a868..5c89c3f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 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.Table;
@@ -43,7 +42,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * Utility client for doing access control admin operations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class AccessControlClient {
   public static final TableName ACL_TABLE_NAME =
       TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl");
@@ -295,4 +293,4 @@ public class AccessControlClient {
     }
     return permList;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java
index f5d16d4..52d10c4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java
@@ -19,10 +19,8 @@
 package org.apache.hadoop.hbase.security.access;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public interface AccessControlConstants {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
index 7bf5304..b25783d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
@@ -27,7 +27,6 @@ import java.util.Map;
 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.hbase.util.Bytes;
 import org.apache.hadoop.io.VersionedWritable;
 
@@ -40,12 +39,10 @@ import com.google.common.collect.Maps;
  * @see TablePermission
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class Permission extends VersionedWritable {
   protected static final byte VERSION = 0;
 
   @InterfaceAudience.Public
-  @InterfaceStability.Evolving
   public enum Action {
     READ('R'), WRITE('W'), EXEC('X'), CREATE('C'), ADMIN('A');
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java
index 5fdeee9..63a4d09 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/Authorizations.java
@@ -22,14 +22,12 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * This class contains visibility labels associated with a Scan/Get deciding which all labeled data
  * current scan/get can access.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class Authorizations {
 
   private List<String> labels;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java
index 765559f..8cd1ae7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/CellVisibility.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.security.visibility;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -28,7 +27,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  * operators AND(&amp;), OR(|) and NOT(!)
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CellVisibility {
 
   private String expression;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java
index d11c167..8d20de8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/InvalidLabelException.java
@@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.security.visibility;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class InvalidLabelException extends DoNotRetryIOException {
   private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java
index 3fbf937..3fb039a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/LabelAlreadyExistsException.java
@@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.security.visibility;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class LabelAlreadyExistsException extends DoNotRetryIOException {
   private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java
index d87bf14..d99f454 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java
@@ -26,7 +26,6 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 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.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Table;
@@ -53,7 +52,6 @@ import com.google.protobuf.ServiceException;
  * Utility client for doing visibility labels admin operations.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class VisibilityClient {
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java
index 90dd0a7..4d87bdf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityControllerNotReadyException.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hbase.security.visibility;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /*
  * This exception indicates that VisibilityController hasn't finished initialization.
  */
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class VisibilityControllerNotReadyException extends IOException {
 
   private static final long serialVersionUID = 1725986525207989173L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a66d4918/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
index d0ef28d..7a1761c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 
 
@@ -27,7 +26,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
  */
 @SuppressWarnings("serial")
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
 public class CorruptedSnapshotException extends HBaseSnapshotException {
 
   /**