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/26 22:52:08 UTC

[01/40] hbase git commit: HBASE-17929 Add more options for PE tool

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 ecdfb8232 -> 177344cdb


HBASE-17929 Add more options for PE tool


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

Branch: refs/heads/hbase-12439
Commit: 3c32032f5ce935eedd2b6d471f20b030c857acbc
Parents: ecdfb82
Author: zhangduo <zh...@apache.org>
Authored: Mon Apr 17 16:20:45 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Apr 18 09:52:34 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/PerformanceEvaluation.java     | 37 ++++++++++++++------
 1 file changed, 26 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3c32032f/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 40e50cf..96ee515 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -636,6 +636,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
     MemoryCompactionPolicy inMemoryCompaction =
         MemoryCompactionPolicy.valueOf(
             CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT);
+    boolean asyncPrefetch = false;
+    boolean cacheBlocks = true;
 
     public TestOptions() {}
 
@@ -1246,8 +1248,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
     @Override
     void testRow(final int i) throws IOException {
-      Scan scan = new Scan(getRandomRow(this.rand, opts.totalRows));
-      scan.setCaching(opts.caching);
+      Scan scan =
+          new Scan().withStartRow(getRandomRow(this.rand, opts.totalRows)).setCaching(opts.caching)
+              .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch);
       FilterList list = new FilterList();
       if (opts.addColumns) {
         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
@@ -1282,8 +1285,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
     @Override
     void testRow(final int i) throws IOException {
       Pair<byte[], byte[]> startAndStopRow = getStartAndStopRow();
-      Scan scan = new Scan(startAndStopRow.getFirst(), startAndStopRow.getSecond());
-      scan.setCaching(opts.caching);
+      Scan scan = new Scan().withStartRow(startAndStopRow.getFirst())
+          .withStopRow(startAndStopRow.getSecond()).setCaching(opts.caching)
+          .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch);
       if (opts.filterAll) {
         scan.setFilter(new FilterAllFilter());
       }
@@ -1477,8 +1481,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
     @Override
     void testRow(final int i) throws IOException {
       if (this.testScanner == null) {
-        Scan scan = new Scan(format(opts.startRow));
-        scan.setCaching(opts.caching);
+        Scan scan = new Scan().withStartRow(format(opts.startRow)).setCaching(opts.caching)
+            .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch);
         if (opts.addColumns) {
           scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
         } else {
@@ -1487,7 +1491,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         if (opts.filterAll) {
           scan.setFilter(new FilterAllFilter());
         }
-       this.testScanner = table.getScanner(scan);
+        this.testScanner = table.getScanner(scan);
       }
       Result r = testScanner.next();
       updateValueSize(r);
@@ -1687,8 +1691,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
       if(opts.filterAll) {
         list.addFilter(new FilterAllFilter());
       }
-      Scan scan = new Scan();
-      scan.setCaching(opts.caching);
+      Scan scan = new Scan().setCaching(opts.caching).setCacheBlocks(opts.cacheBlocks)
+          .setAsyncPrefetch(opts.asyncPrefetch);
       if (opts.addColumns) {
         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
       } else {
@@ -2138,8 +2142,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
       final String inMemoryCompaction = "--inmemoryCompaction=";
       if (cmd.startsWith(inMemoryCompaction)) {
-        opts.inMemoryCompaction = opts.inMemoryCompaction.valueOf(cmd.substring
-            (inMemoryCompaction.length()));
+        opts.inMemoryCompaction =
+            MemoryCompactionPolicy.valueOf(cmd.substring(inMemoryCompaction.length()));
         continue;
       }
 
@@ -2155,6 +2159,17 @@ public class PerformanceEvaluation extends Configured implements Tool {
         continue;
       }
 
+      final String asyncPrefetch = "--asyncPrefetch";
+      if (cmd.startsWith(asyncPrefetch)) {
+        opts.asyncPrefetch = true;
+        continue;
+      }
+
+      final String cacheBlocks = "--cacheBlocks=";
+      if (cmd.startsWith(cacheBlocks)) {
+        opts.cacheBlocks = Boolean.parseBoolean(cmd.substring(cacheBlocks.length()));
+        continue;
+      }
       if (isCommandClass(cmd)) {
         opts.cmdName = cmd;
         try {


[12/40] 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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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..63e63ea 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;
       }
     }
-
   }
 
   /**
@@ -185,54 +190,72 @@ public class TestMemStoreLAB {
    */
   @Test
   public void testLABChunkQueue() throws Exception {
-    MemStoreLABImpl mslab = new MemStoreLABImpl();
-    // by default setting, there should be no chunks initialized in the pool
-    assertTrue(mslab.getPooledChunks().isEmpty());
-    // reset mslab with chunk pool
-    Configuration conf = HBaseConfiguration.create();
-    conf.setDouble(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.1);
-    // 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();
-    mslab = new MemStoreLABImpl(conf);
-    // launch multiple threads to trigger frequent chunk retirement
-    List<Thread> threads = new ArrayList<>();
-    final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
-        new byte[MemStoreLABImpl.MAX_ALLOC_DEFAULT - 24]);
-    for (int i = 0; i < 10; i++) {
-      threads.add(getChunkQueueTestThread(mslab, "testLABChunkQueue-" + i, kv));
-    }
-    for (Thread thread : threads) {
-      thread.start();
-    }
-    // let it run for some time
-    Thread.sleep(1000);
-    for (Thread thread : threads) {
-      thread.interrupt();
-    }
-    boolean threadsRunning = true;
-    while (threadsRunning) {
+    ChunkCreator oldInstance = null;
+    try {
+      MemStoreLABImpl mslab = new MemStoreLABImpl();
+      // by default setting, there should be no chunks initialized in the pool
+      assertTrue(mslab.getPooledChunks().isEmpty());
+      oldInstance = ChunkCreator.INSTANCE;
+      ChunkCreator.INSTANCE = null;
+      // reset mslab with chunk pool
+      Configuration conf = HBaseConfiguration.create();
+      conf.setDouble(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.1);
+      // 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
+      long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
+          .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
+      ChunkCreator.initialize(MemStoreLABImpl.MAX_ALLOC_DEFAULT, false,
+        globalMemStoreLimit, 0.1f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+      ChunkCreator.clearDisableFlag();
+      mslab = new MemStoreLABImpl(conf);
+      // launch multiple threads to trigger frequent chunk retirement
+      List<Thread> threads = new ArrayList<>();
+      final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
+          new byte[MemStoreLABImpl.MAX_ALLOC_DEFAULT - 32]);
+      for (int i = 0; i < 10; i++) {
+        threads.add(getChunkQueueTestThread(mslab, "testLABChunkQueue-" + i, kv));
+      }
       for (Thread thread : threads) {
-        if (thread.isAlive()) {
-          threadsRunning = true;
-          break;
+        thread.start();
+      }
+      // let it run for some time
+      Thread.sleep(1000);
+      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;
         }
       }
-      threadsRunning = false;
+      // none of the chunkIds would have been returned back
+      assertTrue("All the chunks must have been cleared", ChunkCreator.INSTANCE.size() != 0);
+      // close the mslab
+      mslab.close();
+      // 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"
+          + " after mslab closed but actually: " + queueLength,
+        queueLength == 0);
+    } finally {
+      ChunkCreator.INSTANCE = oldInstance;
     }
-    // close the mslab
-    mslab.close();
-    // 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"
-        + " after mslab closed but actually: " + queueLength, queueLength == 0);
   }
 
   private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,
       Cell cellToCopyInto) {
     Thread thread = new Thread() {
-      boolean stopped = false;
+      volatile boolean stopped = false;
 
       @Override
       public void run() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/972e8c8c/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..1af98e9
--- /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() {
+      volatile 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/972e8c8c/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/972e8c8c/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/972e8c8c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
index bf0fb05..c1fd6a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
@@ -60,12 +60,13 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 
 import com.google.common.collect.Lists;
-import org.junit.rules.TestName;
 
 /**
  * Test cases against ReversibleKeyValueScanner
@@ -91,6 +92,10 @@ public class TestReversibleScanners {
   @Rule
   public TestName name = new TestName();
 
+  @BeforeClass
+  public static void setUp() {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+  }
   @Test
   public void testReversibleStoreFileScanner() throws IOException {
     FileSystem fs = TEST_UTIL.getTestFileSystem();

http://git-wip-us.apache.org/repos/asf/hbase/blob/972e8c8c/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 76bf1cc..64e8397 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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
index 0be7b31..09d27af 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
@@ -54,7 +54,9 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
+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.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;

http://git-wip-us.apache.org/repos/asf/hbase/blob/972e8c8c/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);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/972e8c8c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
index 8847c4c..2cf576c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
@@ -37,7 +37,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -153,7 +155,7 @@ public class TestFSHLog extends AbstractTestFSWAL {
           new HTableDescriptor(TableName.valueOf(this.name.getMethodName())).addFamily(new HColumnDescriptor(b));
       HRegionInfo hri =
           new HRegionInfo(htd.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-
+      ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
       final HRegion region = TEST_UTIL.createLocalHRegion(hri, htd, log);
       ExecutorService exec = Executors.newFixedThreadPool(2);
 


[30/40] hbase git commit: HBASE-17952 The new options for PE tool do not work

Posted by sy...@apache.org.
HBASE-17952 The new options for PE tool do not work


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

Branch: refs/heads/hbase-12439
Commit: 72fac379815d79fdeac1452b36cd12fb5492f627
Parents: 49f707f
Author: zhangduo <zh...@apache.org>
Authored: Mon Apr 24 17:00:32 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Apr 25 09:34:49 2017 +0800

----------------------------------------------------------------------
 .../test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/72fac379/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index a3d3254..d0b7319 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -685,6 +685,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
       this.columns = that.columns;
       this.caching = that.caching;
       this.inMemoryCompaction = that.inMemoryCompaction;
+      this.asyncPrefetch = that.asyncPrefetch;
+      this.cacheBlocks = that.cacheBlocks;
+      this.scanReadType = that.scanReadType;
     }
 
     public int getCaching() {


[33/40] hbase git commit: HBASE-15143 Procedure v2 - Web UI displaying queues

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 8ff19b2..e4ce4cb 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
@@ -62144,6 +62144,1133 @@ public final class MasterProtos {
 
   }
 
+  public interface ListLocksRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.ListLocksRequest)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+  }
+  /**
+   * Protobuf type {@code hbase.pb.ListLocksRequest}
+   */
+  public  static final class ListLocksRequest extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.ListLocksRequest)
+      ListLocksRequestOrBuilder {
+    // Use ListLocksRequest.newBuilder() to construct.
+    private ListLocksRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private ListLocksRequest() {
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ListLocksRequest(
+        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();
+      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;
+            }
+          }
+        }
+      } 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.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksRequest_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.Builder.class);
+    }
+
+    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 {
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      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.protobuf.generated.MasterProtos.ListLocksRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest) obj;
+
+      boolean result = true;
+      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();
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest 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;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.ListLocksRequest}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.ListLocksRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequestOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksRequest_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.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();
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest(this);
+        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.protobuf.generated.MasterProtos.ListLocksRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.getDefaultInstance()) return this;
+        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.protobuf.generated.MasterProtos.ListLocksRequest 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.protobuf.generated.MasterProtos.ListLocksRequest) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        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:hbase.pb.ListLocksRequest)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ListLocksRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ListLocksRequest>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<ListLocksRequest>() {
+      public ListLocksRequest 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 ListLocksRequest(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ListLocksRequest> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ListLocksRequest> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface ListLocksResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.ListLocksResponse)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo> 
+        getLockList();
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo getLock(int index);
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    int getLockCount();
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder> 
+        getLockOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder getLockOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.ListLocksResponse}
+   */
+  public  static final class ListLocksResponse extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.ListLocksResponse)
+      ListLocksResponseOrBuilder {
+    // Use ListLocksResponse.newBuilder() to construct.
+    private ListLocksResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private ListLocksResponse() {
+      lock_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ListLocksResponse(
+        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 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                lock_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              lock_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.PARSER, extensionRegistry));
+              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 {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          lock_ = java.util.Collections.unmodifiableList(lock_);
+        }
+        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.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksResponse_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.Builder.class);
+    }
+
+    public static final int LOCK_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo> lock_;
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo> getLockList() {
+      return lock_;
+    }
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder> 
+        getLockOrBuilderList() {
+      return lock_;
+    }
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    public int getLockCount() {
+      return lock_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo getLock(int index) {
+      return lock_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder getLockOrBuilder(
+        int index) {
+      return lock_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      for (int i = 0; i < getLockCount(); i++) {
+        if (!getLock(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      for (int i = 0; i < lock_.size(); i++) {
+        output.writeMessage(1, lock_.get(i));
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < lock_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, lock_.get(i));
+      }
+      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.protobuf.generated.MasterProtos.ListLocksResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse) obj;
+
+      boolean result = true;
+      result = result && getLockList()
+          .equals(other.getLockList());
+      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 (getLockCount() > 0) {
+        hash = (37 * hash) + LOCK_FIELD_NUMBER;
+        hash = (53 * hash) + getLockList().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse 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;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.ListLocksResponse}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.ListLocksResponse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponseOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksResponse_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.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) {
+          getLockFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (lockBuilder_ == null) {
+          lock_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          lockBuilder_.clear();
+        }
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListLocksResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse(this);
+        int from_bitField0_ = bitField0_;
+        if (lockBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            lock_ = java.util.Collections.unmodifiableList(lock_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.lock_ = lock_;
+        } else {
+          result.lock_ = lockBuilder_.build();
+        }
+        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.protobuf.generated.MasterProtos.ListLocksResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.getDefaultInstance()) return this;
+        if (lockBuilder_ == null) {
+          if (!other.lock_.isEmpty()) {
+            if (lock_.isEmpty()) {
+              lock_ = other.lock_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureLockIsMutable();
+              lock_.addAll(other.lock_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.lock_.isEmpty()) {
+            if (lockBuilder_.isEmpty()) {
+              lockBuilder_.dispose();
+              lockBuilder_ = null;
+              lock_ = other.lock_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              lockBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getLockFieldBuilder() : null;
+            } else {
+              lockBuilder_.addAllMessages(other.lock_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getLockCount(); i++) {
+          if (!getLock(i).isInitialized()) {
+            return false;
+          }
+        }
+        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.protobuf.generated.MasterProtos.ListLocksResponse 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.protobuf.generated.MasterProtos.ListLocksResponse) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo> lock_ =
+        java.util.Collections.emptyList();
+      private void ensureLockIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          lock_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo>(lock_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder> lockBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo> getLockList() {
+        if (lockBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(lock_);
+        } else {
+          return lockBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public int getLockCount() {
+        if (lockBuilder_ == null) {
+          return lock_.size();
+        } else {
+          return lockBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo getLock(int index) {
+        if (lockBuilder_ == null) {
+          return lock_.get(index);
+        } else {
+          return lockBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder setLock(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo value) {
+        if (lockBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureLockIsMutable();
+          lock_.set(index, value);
+          onChanged();
+        } else {
+          lockBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder setLock(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder builderForValue) {
+        if (lockBuilder_ == null) {
+          ensureLockIsMutable();
+          lock_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          lockBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder addLock(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo value) {
+        if (lockBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureLockIsMutable();
+          lock_.add(value);
+          onChanged();
+        } else {
+          lockBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder addLock(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo value) {
+        if (lockBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureLockIsMutable();
+          lock_.add(index, value);
+          onChanged();
+        } else {
+          lockBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder addLock(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder builderForValue) {
+        if (lockBuilder_ == null) {
+          ensureLockIsMutable();
+          lock_.add(builderForValue.build());
+          onChanged();
+        } else {
+          lockBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder addLock(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder builderForValue) {
+        if (lockBuilder_ == null) {
+          ensureLockIsMutable();
+          lock_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          lockBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder addAllLock(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo> values) {
+        if (lockBuilder_ == null) {
+          ensureLockIsMutable();
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, lock_);
+          onChanged();
+        } else {
+          lockBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder clearLock() {
+        if (lockBuilder_ == null) {
+          lock_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          lockBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public Builder removeLock(int index) {
+        if (lockBuilder_ == null) {
+          ensureLockIsMutable();
+          lock_.remove(index);
+          onChanged();
+        } else {
+          lockBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder getLockBuilder(
+          int index) {
+        return getLockFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder getLockOrBuilder(
+          int index) {
+        if (lockBuilder_ == null) {
+          return lock_.get(index);  } else {
+          return lockBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder> 
+           getLockOrBuilderList() {
+        if (lockBuilder_ != null) {
+          return lockBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(lock_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder addLockBuilder() {
+        return getLockFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder addLockBuilder(
+          int index) {
+        return getLockFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.LockInfo lock = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder> 
+           getLockBuilderList() {
+        return getLockFieldBuilder().getBuilderList();
+      }
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder> 
+          getLockFieldBuilder() {
+        if (lockBuilder_ == null) {
+          lockBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder>(
+                  lock_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          lock_ = null;
+        }
+        return lockBuilder_;
+      }
+      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:hbase.pb.ListLocksResponse)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ListLocksResponse)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ListLocksResponse>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<ListLocksResponse>() {
+      public ListLocksResponse 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 ListLocksResponse(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ListLocksResponse> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ListLocksResponse> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
   public interface SetQuotaRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.SetQuotaRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
@@ -70841,6 +71968,14 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse> done);
 
       /**
+       * <code>rpc ListLocks(.hbase.pb.ListLocksRequest) returns (.hbase.pb.ListLocksResponse);</code>
+       */
+      public abstract void listLocks(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse> done);
+
+      /**
        * <pre>
        ** Add a replication peer 
        * </pre>
@@ -71446,6 +72581,14 @@ public final class MasterProtos {
         }
 
         @java.lang.Override
+        public  void listLocks(
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse> done) {
+          impl.listLocks(controller, request, done);
+        }
+
+        @java.lang.Override
         public  void addReplicationPeer(
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest request,
@@ -71668,24 +72811,26 @@ public final class MasterProtos {
             case 59:
               return impl.listProcedures(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest)request);
             case 60:
-              return impl.addReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest)request);
+              return impl.listLocks(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest)request);
             case 61:
-              return impl.removeReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest)request);
+              return impl.addReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest)request);
             case 62:
-              return impl.enableReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest)request);
+              return impl.removeReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest)request);
             case 63:
-              return impl.disableReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest)request);
+              return impl.enableReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest)request);
             case 64:
-              return impl.getReplicationPeerConfig(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest)request);
+              return impl.disableReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest)request);
             case 65:
-              return impl.updateReplicationPeerConfig(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest)request);
+              return impl.getReplicationPeerConfig(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest)request);
             case 66:
-              return impl.listReplicationPeers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest)request);
+              return impl.updateReplicationPeerConfig(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest)request);
             case 67:
-              return impl.listDrainingRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest)request);
+              return impl.listReplicationPeers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest)request);
             case 68:
-              return impl.drainRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest)request);
+              return impl.listDrainingRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest)request);
             case 69:
+              return impl.drainRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest)request);
+            case 70:
               return impl.removeDrainFromRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -71822,24 +72967,26 @@ public final class MasterProtos {
             case 59:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest.getDefaultInstance();
             case 60:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.getDefaultInstance();
             case 61:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.getDefaultInstance();
             case 62:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest.getDefaultInstance();
             case 63:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest.getDefaultInstance();
             case 64:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest.getDefaultInstance();
             case 65:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest.getDefaultInstance();
             case 66:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest.getDefaultInstance();
             case 67:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest.getDefaultInstance();
             case 68:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
             case 69:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+            case 70:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -71976,24 +73123,26 @@ public final class MasterProtos {
             case 59:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance();
             case 60:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.getDefaultInstance();
             case 61:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse.getDefaultInstance();
             case 62:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse.getDefaultInstance();
             case 63:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse.getDefaultInstance();
             case 64:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse.getDefaultInstance();
             case 65:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse.getDefaultInstance();
             case 66:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance();
             case 67:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance();
             case 68:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
             case 69:
+              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+            case 70:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -72755,6 +73904,14 @@ public final class MasterProtos {
         org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse> done);
 
     /**
+     * <code>rpc ListLocks(.hbase.pb.ListLocksRequest) returns (.hbase.pb.ListLocksResponse);</code>
+     */
+    public abstract void listLocks(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest request,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse> done);
+
+    /**
      * <pre>
      ** Add a replication peer 
      * </pre>
@@ -73197,51 +74354,56 @@ public final class MasterProtos {
               done));
           return;
         case 60:
+          this.listLocks(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest)request,
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse>specializeCallback(
+              done));
+          return;
+        case 61:
           this.addReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse>specializeCallback(
               done));
           return;
-        case 61:
+        case 62:
           this.removeReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse>specializeCallback(
               done));
           return;
-        case 62:
+        case 63:
           this.enableReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse>specializeCallback(
               done));
           return;
-        case 63:
+        case 64:
           this.disableReplicationPeer(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse>specializeCallback(
               done));
           return;
-        case 64:
+        case 65:
           this.getReplicationPeerConfig(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse>specializeCallback(
               done));
           return;
-        case 65:
+        case 66:
           this.updateReplicationPeerConfig(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse>specializeCallback(
               done));
           return;
-        case 66:
+        case 67:
           this.listReplicationPeers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse>specializeCallback(
               done));
           return;
-        case 67:
+        case 68:
           this.listDrainingRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse>specializeCallback(
               done));
           return;
-        case 68:
+        case 69:
           this.drainRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse>specializeCallback(
               done));
           return;
-        case 69:
+        case 70:
           this.removeDrainFromRegionServers(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse>specializeCallback(
               done));
@@ -73381,24 +74543,26 @@ public final class MasterProtos {
         case 59:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest.getDefaultInstance();
         case 60:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest.getDefaultInstance();
         case 61:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest.getDefaultInstance();
         case 62:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest.getDefaultInstance();
         case 63:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest.getDefaultInstance();
         case 64:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest.getDefaultInstance();
         case 65:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest.getDefaultInstance();
         case 66:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest.getDefaultInstance();
         case 67:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest.getDefaultInstance();
         case 68:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest.getDefaultInstance();
         case 69:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest.getDefaultInstance();
+        case 70:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -73535,24 +74699,26 @@ public final class MasterProtos {
         case 59:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance();
         case 60:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.getDefaultInstance();
         case 61:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse.getDefaultInstance();
         case 62:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse.getDefaultInstance();
         case 63:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse.getDefaultInstance();
         case 64:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse.getDefaultInstance();
         case 65:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse.getDefaultInstance();
         case 66:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance();
         case 67:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance();
         case 68:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance();
         case 69:
+          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance();
+        case 70:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -74475,12 +75641,27 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance()));
       }
 
+      public  void listLocks(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest request,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(60),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.getDefaultInstance(),
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.class,
+            org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.getDefaultInstance()));
+      }
+
       public  void addReplicationPeer(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(60),
+          getDescriptor().getMethods().get(61),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse.getDefaultInstance(),
@@ -74495,7 +75676,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(61),
+          getDescriptor().getMethods().get(62),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse.getDefaultInstance(),
@@ -74510,7 +75691,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(62),
+          getDescriptor().getMethods().get(63),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse.getDefaultInstance(),
@@ -74525,7 +75706,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(63),
+          getDescriptor().getMethods().get(64),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse.getDefaultInstance(),
@@ -74540,7 +75721,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(64),
+          getDescriptor().getMethods().get(65),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse.getDefaultInstance(),
@@ -74555,7 +75736,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(65),
+          getDescriptor().getMethods().get(66),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse.getDefaultInstance(),
@@ -74570,7 +75751,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(66),
+          getDescriptor().getMethods().get(67),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse.getDefaultInstance(),
@@ -74585,7 +75766,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(67),
+          getDescriptor().getMethods().get(68),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse.getDefaultInstance(),
@@ -74600,7 +75781,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(68),
+          getDescriptor().getMethods().get(69),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse.getDefaultInstance(),
@@ -74615,7 +75796,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(69),
+          getDescriptor().getMethods().get(70),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse.getDefaultInstance(),
@@ -74932,6 +76113,11 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse listLocks(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse addReplicationPeer(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest request)
@@ -75710,12 +76896,24 @@ public final class MasterProtos {
       }
 
 
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse listLocks(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest request)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(60),
+          controller,
+          request,
+          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse.getDefaultInstance());
+      }
+
+
       public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse addReplicationPeer(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(60),
+          getDescriptor().getMethods().get(61),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse.getDefaultInstance());
@@ -75727,7 +76925,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(61),
+          getDescriptor().getMethods().get(62),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse.getDefaultInstance());
@@ -75739,7 +76937,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(62),
+          getDescriptor().getMethods().get(63),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse.getDefaultInstance());
@@ -75751,7 +76949,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(63),
+          getDescriptor().getMethods().get(64),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse.getDefaultInstance());
@@ -75763,7 +76961,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(64),
+          getDescriptor().getMethods().get(65),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse.getDefaultInstance());
@@ -75775,7 +76973,7 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(65),
+          getDescriptor().getMethods().get(66),
           controlle

<TRUNCATED>

[10/40] hbase git commit: HBASE-17914 Create a new reader instead of cloning a new StoreFile when compaction

Posted by sy...@apache.org.
HBASE-17914 Create a new reader instead of cloning a new StoreFile when compaction


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

Branch: refs/heads/hbase-12439
Commit: 66b616d7a3d6f4ad6d20962e2dfc0c82a4092ddb
Parents: 719a30b
Author: zhangduo <zh...@apache.org>
Authored: Mon Apr 17 22:53:49 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 19 09:26:33 2017 +0800

----------------------------------------------------------------------
 .../hbase/io/FSDataInputStreamWrapper.java      |  63 +++---
 .../org/apache/hadoop/hbase/io/FileLink.java    |  14 +-
 .../hadoop/hbase/io/HalfStoreFileReader.java    |  13 +-
 .../hadoop/hbase/io/hfile/CacheConfig.java      |   9 +-
 .../org/apache/hadoop/hbase/io/hfile/HFile.java |  85 ++++----
 .../hbase/io/hfile/HFilePrettyPrinter.java      |   2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  26 +--
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |  45 ++--
 .../procedure/MergeTableRegionsProcedure.java   |   9 +-
 .../procedure/SplitTableRegionProcedure.java    |   8 +-
 .../apache/hadoop/hbase/mob/CachedMobFile.java  |   4 +-
 .../org/apache/hadoop/hbase/mob/MobFile.java    |   8 +-
 .../org/apache/hadoop/hbase/mob/MobUtils.java   |  13 +-
 .../compactions/PartitionedMobCompactor.java    |  26 +--
 .../regionserver/DefaultStoreFileManager.java   |   2 +-
 .../hadoop/hbase/regionserver/HMobStore.java    |   6 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   4 +-
 .../hbase/regionserver/HRegionFileSystem.java   |   6 +-
 .../hadoop/hbase/regionserver/HStore.java       |  19 +-
 .../regionserver/ReversedStoreScanner.java      |   2 +-
 .../hadoop/hbase/regionserver/StoreFile.java    | 216 ++++++++++++-------
 .../hbase/regionserver/StoreFileInfo.java       |  21 +-
 .../hbase/regionserver/StoreFileReader.java     |  86 ++++----
 .../hbase/regionserver/StoreFileScanner.java    |  50 +++--
 .../hadoop/hbase/regionserver/StoreScanner.java |   6 +-
 .../regionserver/compactions/Compactor.java     |  44 +---
 .../hadoop/hbase/util/CompressionTest.java      |   2 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   6 +-
 .../hbase/util/hbck/HFileCorruptionChecker.java |   4 +-
 .../hbase/HFilePerformanceEvaluation.java       |   2 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   1 +
 .../hbase/io/TestHalfStoreFileReader.java       | 192 ++++++++---------
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |   2 +-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |   8 +-
 .../hbase/io/hfile/TestHFileBlockIndex.java     |   6 +-
 .../hbase/io/hfile/TestHFileEncryption.java     |   6 +-
 .../TestHFileInlineToRootChunkConversion.java   |   2 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |   2 +-
 .../hadoop/hbase/io/hfile/TestReseekTo.java     |   4 +-
 .../hfile/TestSeekBeforeWithInlineBlocks.java   |   2 +-
 .../hadoop/hbase/io/hfile/TestSeekTo.java       |   8 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java |  10 +-
 .../TestImportTSVWithVisibilityLabels.java      |   2 +-
 .../hadoop/hbase/mapreduce/TestImportTsv.java   |   2 +-
 .../mapreduce/TestLoadIncrementalHFiles.java    |   4 +-
 .../apache/hadoop/hbase/mob/TestMobFile.java    |   8 +-
 .../hbase/mob/compactions/TestMobCompactor.java |   9 +-
 .../TestPartitionedMobCompactor.java            |  18 +-
 .../regionserver/DataBlockEncodingTool.java     |   7 +-
 .../EncodedSeekPerformanceTest.java             |  12 +-
 .../hbase/regionserver/MockStoreFile.java       |  25 ++-
 .../regionserver/TestCacheOnWriteInSchema.java  |   6 +-
 .../regionserver/TestCompactionPolicy.java      |   3 -
 .../regionserver/TestCompoundBloomFilter.java   |   7 +-
 .../regionserver/TestEncryptionKeyRotation.java |   2 +-
 .../TestEncryptionRandomKeying.java             |   2 +-
 .../hbase/regionserver/TestFSErrorsExposed.java |  12 +-
 .../regionserver/TestMobStoreCompaction.java    |   7 +-
 .../regionserver/TestReversibleScanners.java    |  33 ++-
 .../hadoop/hbase/regionserver/TestStore.java    |   2 +-
 .../hbase/regionserver/TestStoreFile.java       | 120 ++++++-----
 .../TestStoreFileScannerWithTagCompression.java |  10 +-
 .../regionserver/compactions/TestCompactor.java |   3 -
 .../compactions/TestStripeCompactionPolicy.java |   3 -
 .../hbase/util/TestHBaseFsckEncryption.java     |   2 +-
 .../hadoop/hbase/spark/BulkLoadSuite.scala      |   8 +-
 66 files changed, 701 insertions(+), 650 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
index b06be6b..055e46a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
@@ -17,11 +17,14 @@
  */
 package org.apache.hadoop.hbase.io;
 
+import java.io.Closeable;
 import java.io.IOException;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -31,11 +34,14 @@ import com.google.common.annotations.VisibleForTesting;
  * as well as closing streams. Initialization is not thread-safe, but normal operation is;
  * see method comments.
  */
-public class FSDataInputStreamWrapper {
+@InterfaceAudience.Private
+public class FSDataInputStreamWrapper implements Closeable {
   private final HFileSystem hfs;
   private final Path path;
   private final FileLink link;
   private final boolean doCloseStreams;
+  private final boolean dropBehind;
+  private final long readahead;
 
   /** Two stream handles, one with and one without FS-level checksum.
    * HDFS checksum setting is on FS level, not single read level, so you have to keep two
@@ -75,43 +81,52 @@ public class FSDataInputStreamWrapper {
   private volatile int hbaseChecksumOffCount = -1;
 
   public FSDataInputStreamWrapper(FileSystem fs, Path path) throws IOException {
-    this(fs, null, path, false);
+    this(fs, path, false, -1L);
   }
 
-  public FSDataInputStreamWrapper(FileSystem fs, Path path, boolean dropBehind) throws IOException {
-    this(fs, null, path, dropBehind);
+  public FSDataInputStreamWrapper(FileSystem fs, Path path, boolean dropBehind, long readahead) throws IOException {
+    this(fs, null, path, dropBehind, readahead);
   }
 
-  public FSDataInputStreamWrapper(FileSystem fs, FileLink link) throws IOException {
-    this(fs, link, null, false);
-  }
   public FSDataInputStreamWrapper(FileSystem fs, FileLink link,
-                                  boolean dropBehind) throws IOException {
-    this(fs, link, null, dropBehind);
+                                  boolean dropBehind, long readahead) throws IOException {
+    this(fs, link, null, dropBehind, readahead);
   }
 
-  private FSDataInputStreamWrapper(FileSystem fs, FileLink link,
-                                   Path path, boolean dropBehind) throws IOException {
+  private FSDataInputStreamWrapper(FileSystem fs, FileLink link, Path path, boolean dropBehind,
+      long readahead) throws IOException {
     assert (path == null) != (link == null);
     this.path = path;
     this.link = link;
     this.doCloseStreams = true;
+    this.dropBehind = dropBehind;
+    this.readahead = readahead;
     // If the fs is not an instance of HFileSystem, then create an instance of HFileSystem
     // that wraps over the specified fs. In this case, we will not be able to avoid
     // checksumming inside the filesystem.
-    this.hfs = (fs instanceof HFileSystem) ? (HFileSystem)fs : new HFileSystem(fs);
+    this.hfs = (fs instanceof HFileSystem) ? (HFileSystem) fs : new HFileSystem(fs);
 
     // Initially we are going to read the tail block. Open the reader w/FS checksum.
     this.useHBaseChecksumConfigured = this.useHBaseChecksum = false;
     this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
+    setStreamOptions(stream);
+  }
+
+  private void setStreamOptions(FSDataInputStream in) {
     try {
       this.stream.setDropBehind(dropBehind);
     } catch (Exception e) {
       // Skipped.
     }
+    if (readahead >= 0) {
+      try {
+        this.stream.setReadahead(readahead);
+      } catch (Exception e) {
+        // Skipped.
+      }
+    }
   }
 
-
   /**
    * Prepares the streams for block reader. NOT THREAD SAFE. Must be called once, after any
    * reads finish and before any other reads start (what happens in reality is we read the
@@ -127,6 +142,7 @@ public class FSDataInputStreamWrapper {
     if (useHBaseChecksum) {
       FileSystem fsNc = hfs.getNoChecksumFs();
       this.streamNoFsChecksum = (link != null) ? link.open(fsNc) : fsNc.open(path);
+      setStreamOptions(streamNoFsChecksum);
       this.useHBaseChecksumConfigured = this.useHBaseChecksum = useHBaseChecksum;
       // Close the checksum stream; we will reopen it if we get an HBase checksum failure.
       this.stream.close();
@@ -150,6 +166,8 @@ public class FSDataInputStreamWrapper {
     link = null;
     hfs = null;
     useHBaseChecksumConfigured = useHBaseChecksum = false;
+    dropBehind = false;
+    readahead = 0;
   }
 
   /**
@@ -201,19 +219,14 @@ public class FSDataInputStreamWrapper {
   }
 
   /** Close stream(s) if necessary. */
-  public void close() throws IOException {
-    if (!doCloseStreams) return;
-    try {
-      if (stream != streamNoFsChecksum && streamNoFsChecksum != null) {
-        streamNoFsChecksum.close();
-        streamNoFsChecksum = null;
-      }
-    } finally {
-      if (stream != null) {
-        stream.close();
-        stream = null;
-      }
+  @Override
+  public void close() {
+    if (!doCloseStreams) {
+      return;
     }
+    // we do not care about the close exception as it is for reading, no data loss issue.
+    IOUtils.closeQuietly(streamNoFsChecksum);
+    IOUtils.closeQuietly(stream);
   }
 
   public HFileSystem getHfs() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
index ca0dfbc..8a79efb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
@@ -29,6 +29,8 @@ 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.fs.CanSetDropBehind;
+import org.apache.hadoop.fs.CanSetReadahead;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileStatus;
@@ -99,7 +101,7 @@ public class FileLink {
    * and the alternative locations, when the file is moved.
    */
   private static class FileLinkInputStream extends InputStream
-      implements Seekable, PositionedReadable {
+      implements Seekable, PositionedReadable, CanSetDropBehind, CanSetReadahead {
     private FSDataInputStream in = null;
     private Path currentPath = null;
     private long pos = 0;
@@ -306,6 +308,16 @@ public class FileLink {
       }
       throw new FileNotFoundException("Unable to open link: " + fileLink);
     }
+
+    @Override
+    public void setReadahead(Long readahead) throws IOException, UnsupportedOperationException {
+      in.setReadahead(readahead);
+    }
+
+    @Override
+    public void setDropBehind(Boolean dropCache) throws IOException, UnsupportedOperationException {
+      in.setDropBehind(dropCache);
+    }
   }
 
   private Path[] locations = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
index a4a281e..c4dbc39 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -72,10 +73,10 @@ public class HalfStoreFileReader extends StoreFileReader {
    * @param conf Configuration
    * @throws IOException
    */
-  public HalfStoreFileReader(final FileSystem fs, final Path p,
-      final CacheConfig cacheConf, final Reference r, final Configuration conf)
+  public HalfStoreFileReader(FileSystem fs, Path p, CacheConfig cacheConf, Reference r,
+      boolean isPrimaryReplicaStoreFile, AtomicInteger refCount, boolean shared, Configuration conf)
       throws IOException {
-    super(fs, p, cacheConf, conf);
+    super(fs, p, cacheConf, isPrimaryReplicaStoreFile, refCount, shared, conf);
     // This is not actual midkey for this half-file; its just border
     // around which we split top and bottom.  Have to look in files to find
     // actual last and first keys for bottom and top halves.  Half-files don't
@@ -99,9 +100,9 @@ public class HalfStoreFileReader extends StoreFileReader {
    * @throws IOException
    */
   public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
-      long size, final CacheConfig cacheConf,  final Reference r, final Configuration conf)
-      throws IOException {
-    super(fs, p, in, size, cacheConf, conf);
+      long size, final CacheConfig cacheConf, final Reference r, boolean isPrimaryReplicaStoreFile,
+      AtomicInteger refCount, boolean shared, final Configuration conf) throws IOException {
+    super(fs, p, in, size, cacheConf, isPrimaryReplicaStoreFile, refCount, shared, conf);
     // This is not actual midkey for this half-file; its just border
     // around which we split top and bottom.  Have to look in files to find
     // actual last and first keys for bottom and top halves.  Half-files don't

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
index 4db60b5..791445b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
@@ -283,11 +283,10 @@ public class CacheConfig {
   }
 
   /**
-   * Create a block cache configuration with the specified cache and
-   * configuration parameters.
+   * Create a block cache configuration with the specified cache and configuration parameters.
    * @param blockCache reference to block cache, null if completely disabled
    * @param cacheDataOnRead whether DATA blocks should be cached on read (we always cache INDEX
-   * blocks and BLOOM blocks; this cannot be disabled).
+   *          blocks and BLOOM blocks; this cannot be disabled).
    * @param inMemory whether blocks should be flagged as in-memory
    * @param cacheDataOnWrite whether data blocks should be cached on write
    * @param cacheIndexesOnWrite whether index blocks should be cached on write
@@ -296,7 +295,9 @@ public class CacheConfig {
    * @param cacheDataCompressed whether to store blocks as compressed in the cache
    * @param prefetchOnOpen whether to prefetch blocks upon open
    * @param cacheDataInL1 If more than one cache tier deployed, if true, cache this column families
-   * data blocks up in the L1 tier.
+   *          data blocks up in the L1 tier.
+   * @param dropBehindCompaction indicate that we should set drop behind to true when open a store
+   *          file reader for compaction
    */
   CacheConfig(final BlockCache blockCache,
       final boolean cacheDataOnRead, final boolean inMemory,

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index c5b334a..0887ee8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -36,6 +36,7 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.LongAdder;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -462,8 +463,6 @@ public class HFile {
 
     boolean isPrimaryReplicaReader();
 
-    void setPrimaryReplicaReader(boolean isPrimaryReplicaReader);
-
     boolean shouldIncludeMemstoreTS();
 
     boolean isDecodeMemstoreTS();
@@ -486,33 +485,32 @@ public class HFile {
    * @param size max size of the trailer.
    * @param cacheConf Cache configuation values, cannot be null.
    * @param hfs
+   * @param primaryReplicaReader true if this is a reader for primary replica
    * @return an appropriate instance of HFileReader
    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
       justification="Intentional")
-  private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis,
-      long size, CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException {
+  private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis, long size,
+      CacheConfig cacheConf, HFileSystem hfs, boolean primaryReplicaReader, Configuration conf)
+      throws IOException {
     FixedFileTrailer trailer = null;
     try {
       boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum();
       assert !isHBaseChecksum; // Initially we must read with FS checksum.
       trailer = FixedFileTrailer.readFromStream(fsdis.getStream(isHBaseChecksum), size);
       switch (trailer.getMajorVersion()) {
-      case 2:
-        LOG.debug("Opening HFile v2 with v3 reader");
-        // Fall through. FindBugs: SF_SWITCH_FALLTHROUGH
-      case 3 :
-        return new HFileReaderImpl(path, trailer, fsdis, size, cacheConf, hfs, conf);
-      default:
-        throw new IllegalArgumentException("Invalid HFile version " + trailer.getMajorVersion());
+        case 2:
+          LOG.debug("Opening HFile v2 with v3 reader");
+          // Fall through. FindBugs: SF_SWITCH_FALLTHROUGH
+        case 3:
+          return new HFileReaderImpl(path, trailer, fsdis, size, cacheConf, hfs,
+              primaryReplicaReader, conf);
+        default:
+          throw new IllegalArgumentException("Invalid HFile version " + trailer.getMajorVersion());
       }
     } catch (Throwable t) {
-      try {
-        fsdis.close();
-      } catch (Throwable t2) {
-        LOG.warn("Error closing fsdis FSDataInputStreamWrapper", t2);
-      }
+      IOUtils.closeQuietly(fsdis);
       throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, t);
     }
   }
@@ -523,13 +521,13 @@ public class HFile {
    * @param fsdis a stream of path's file
    * @param size max size of the trailer.
    * @param cacheConf Cache configuration for hfile's contents
+   * @param primaryReplicaReader true if this is a reader for primary replica
    * @param conf Configuration
    * @return A version specific Hfile Reader
    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
    */
-  @SuppressWarnings("resource")
-  public static Reader createReader(FileSystem fs, Path path,
-      FSDataInputStreamWrapper fsdis, long size, CacheConfig cacheConf, Configuration conf)
+  public static Reader createReader(FileSystem fs, Path path, FSDataInputStreamWrapper fsdis,
+      long size, CacheConfig cacheConf, boolean primaryReplicaReader, Configuration conf)
       throws IOException {
     HFileSystem hfs = null;
 
@@ -540,9 +538,9 @@ public class HFile {
     if (!(fs instanceof HFileSystem)) {
       hfs = new HFileSystem(fs);
     } else {
-      hfs = (HFileSystem)fs;
+      hfs = (HFileSystem) fs;
     }
-    return pickReaderVersion(path, fsdis, size, cacheConf, hfs, conf);
+    return pickReaderVersion(path, fsdis, size, cacheConf, hfs, primaryReplicaReader, conf);
   }
 
   /**
@@ -553,35 +551,39 @@ public class HFile {
   * @throws IOException Will throw a CorruptHFileException
   * (DoNotRetryIOException subtype) if hfile is corrupt/invalid.
   */
- public static Reader createReader(
-     FileSystem fs, Path path,  Configuration conf) throws IOException {
-     return createReader(fs, path, CacheConfig.DISABLED, conf);
- }
+  public static Reader createReader(FileSystem fs, Path path, Configuration conf)
+      throws IOException {
+    // The primaryReplicaReader is mainly used for constructing block cache key, so if we do not use
+    // block cache then it is OK to set it as any value. We use true here.
+    return createReader(fs, path, CacheConfig.DISABLED, true, conf);
+  }
 
   /**
-   *
    * @param fs filesystem
    * @param path Path to file to read
-   * @param cacheConf This must not be null.  @see {@link org.apache.hadoop.hbase.io.hfile.CacheConfig#CacheConfig(Configuration)}
+   * @param cacheConf This must not be null. @see
+   *          {@link org.apache.hadoop.hbase.io.hfile.CacheConfig#CacheConfig(Configuration)}
+   * @param primaryReplicaReader true if this is a reader for primary replica
    * @return an active Reader instance
-   * @throws IOException Will throw a CorruptHFileException (DoNotRetryIOException subtype) if hfile is corrupt/invalid.
+   * @throws IOException Will throw a CorruptHFileException (DoNotRetryIOException subtype) if hfile
+   *           is corrupt/invalid.
    */
-  public static Reader createReader(
-      FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf) throws IOException {
+  public static Reader createReader(FileSystem fs, Path path, CacheConfig cacheConf,
+      boolean primaryReplicaReader, Configuration conf) throws IOException {
     Preconditions.checkNotNull(cacheConf, "Cannot create Reader with null CacheConf");
     FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fs, path);
-    return pickReaderVersion(path, stream, fs.getFileStatus(path).getLen(),
-      cacheConf, stream.getHfs(), conf);
+    return pickReaderVersion(path, stream, fs.getFileStatus(path).getLen(), cacheConf,
+      stream.getHfs(), primaryReplicaReader, conf);
   }
 
   /**
    * This factory method is used only by unit tests
    */
-  static Reader createReaderFromStream(Path path,
-      FSDataInputStream fsdis, long size, CacheConfig cacheConf, Configuration conf)
-      throws IOException {
+  @VisibleForTesting
+  static Reader createReaderFromStream(Path path, FSDataInputStream fsdis, long size,
+      CacheConfig cacheConf, Configuration conf) throws IOException {
     FSDataInputStreamWrapper wrapper = new FSDataInputStreamWrapper(fsdis);
-    return pickReaderVersion(path, wrapper, size, cacheConf, null, conf);
+    return pickReaderVersion(path, wrapper, size, cacheConf, null, true, conf);
   }
 
   /**
@@ -606,22 +608,13 @@ public class HFile {
       throws IOException {
     final Path path = fileStatus.getPath();
     final long size = fileStatus.getLen();
-    FSDataInputStreamWrapper fsdis = new FSDataInputStreamWrapper(fs, path);
-    try {
+    try (FSDataInputStreamWrapper fsdis = new FSDataInputStreamWrapper(fs, path)) {
       boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum();
       assert !isHBaseChecksum; // Initially we must read with FS checksum.
       FixedFileTrailer.readFromStream(fsdis.getStream(isHBaseChecksum), size);
       return true;
     } catch (IllegalArgumentException e) {
       return false;
-    } catch (IOException e) {
-      throw e;
-    } finally {
-      try {
-        fsdis.close();
-      } catch (Throwable t) {
-        LOG.warn("Error closing fsdis FSDataInputStreamWrapper: " + path, t);
-      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index 030a25e..43b5c24 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -306,7 +306,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       return -2;
     }
 
-    HFile.Reader reader = HFile.createReader(fs, file, new CacheConfig(getConf()), getConf());
+    HFile.Reader reader = HFile.createReader(fs, file, new CacheConfig(getConf()), true, getConf());
 
     Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 4e8cbaa..f0a1fa1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -85,7 +85,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
   /** Filled when we read in the trailer. */
   private final Compression.Algorithm compressAlgo;
 
-  private boolean isPrimaryReplicaReader;
+  private final boolean primaryReplicaReader;
 
   /**
    * What kind of data block encoding should be used while reading, writing,
@@ -156,6 +156,14 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
   /** Minor versions starting with this number have faked index key */
   static final int MINOR_VERSION_WITH_FAKED_KEY = 3;
 
+  @VisibleForTesting
+  @Deprecated
+  public HFileReaderImpl(Path path, FixedFileTrailer trailer, FSDataInputStreamWrapper fsdis,
+      long fileSize, CacheConfig cacheConf, HFileSystem hfs, Configuration conf)
+      throws IOException {
+    this(path, trailer, fsdis, fileSize, cacheConf, hfs, true, conf);
+  }
+
   /**
    * Opens a HFile. You must load the index before you can use it by calling
    * {@link #loadFileInfo()}.
@@ -175,11 +183,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    *          Configuration
    */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
-  public HFileReaderImpl(final Path path, FixedFileTrailer trailer,
-      final FSDataInputStreamWrapper fsdis,
-      final long fileSize, final CacheConfig cacheConf, final HFileSystem hfs,
-      final Configuration conf)
-  throws IOException {
+  public HFileReaderImpl(Path path, FixedFileTrailer trailer, FSDataInputStreamWrapper fsdis,
+      long fileSize, CacheConfig cacheConf, HFileSystem hfs, boolean primaryReplicaReader,
+      Configuration conf) throws IOException {
     this.trailer = trailer;
     this.compressAlgo = trailer.getCompressionCodec();
     this.cacheConf = cacheConf;
@@ -187,6 +193,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     this.path = path;
     this.name = path.getName();
     this.conf = conf;
+    this.primaryReplicaReader = primaryReplicaReader;
     checkFileVersion();
     this.hfileContext = createHFileContext(fsdis, fileSize, hfs, path, trailer);
     this.fsBlockReader = new HFileBlock.FSReaderImpl(fsdis, fileSize, hfs, path, hfileContext);
@@ -453,12 +460,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
   @Override
   public boolean isPrimaryReplicaReader() {
-    return isPrimaryReplicaReader;
-  }
-
-  @Override
-  public void setPrimaryReplicaReader(boolean isPrimaryReplicaReader) {
-    this.isPrimaryReplicaReader = isPrimaryReplicaReader;
+    return primaryReplicaReader;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/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 19daeed..3af4290 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
@@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.mapreduce;
 
 import static java.lang.String.format;
 
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -27,7 +32,6 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -63,9 +67,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-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.client.Admin;
 import org.apache.hadoop.hbase.client.ClientServiceCallable;
@@ -99,10 +100,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Multimaps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 /**
  * Tool to load the output of HFileOutputFormat into an existing table.
  */
@@ -937,8 +934,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     }
     HFile.Reader hfr = null;
     try {
-      hfr = HFile.createReader(fs, hfilePath,
-          new CacheConfig(getConf()), getConf());
+      hfr = HFile.createReader(fs, hfilePath, new CacheConfig(getConf()), true, getConf());
     } catch (FileNotFoundException fnfe) {
       LOG.debug("encountered", fnfe);
       return new Pair<>(null, hfilePath.getName());
@@ -1105,7 +1101,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     HalfStoreFileReader halfReader = null;
     StoreFileWriter halfWriter = null;
     try {
-      halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, conf);
+      halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, true,
+          new AtomicInteger(0), true, conf);
       Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
 
       int blocksize = familyDescriptor.getBlocksize();
@@ -1213,30 +1210,26 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
       public void bulkHFile(final HColumnDescriptor hcd, final FileStatus hfileStatus)
           throws IOException {
         Path hfile = hfileStatus.getPath();
-        HFile.Reader reader = HFile.createReader(fs, hfile,
-            new CacheConfig(getConf()), getConf());
-        try {
+        try (HFile.Reader reader =
+            HFile.createReader(fs, hfile, new CacheConfig(getConf()), true, getConf())) {
           if (hcd.getCompressionType() != reader.getFileContext().getCompression()) {
             hcd.setCompressionType(reader.getFileContext().getCompression());
-            LOG.info("Setting compression " + hcd.getCompressionType().name() +
-                     " for family " + hcd.toString());
+            LOG.info("Setting compression " + hcd.getCompressionType().name() + " for family " +
+                hcd.toString());
           }
           reader.loadFileInfo();
           byte[] first = reader.getFirstRowKey();
-          byte[] last  = reader.getLastRowKey();
+          byte[] last = reader.getLastRowKey();
 
-          LOG.info("Trying to figure out region boundaries hfile=" + hfile +
-            " first=" + Bytes.toStringBinary(first) +
-            " last="  + Bytes.toStringBinary(last));
+          LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first=" +
+              Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
 
           // To eventually infer start key-end key boundaries
-          Integer value = map.containsKey(first)? map.get(first):0;
-          map.put(first, value+1);
+          Integer value = map.containsKey(first) ? map.get(first) : 0;
+          map.put(first, value + 1);
 
-          value = map.containsKey(last)? map.get(last):0;
-          map.put(last, value-1);
-        } finally {
-          reader.close();
+          value = map.containsKey(last) ? map.get(last) : 0;
+          map.put(last, value - 1);
         }
       }
     });

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
index 366378a..3600fe0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
@@ -264,7 +264,7 @@ public class MergeTableRegionsProcedure
 
   @Override
   protected MergeTableRegionsState getState(final int stateId) {
-    return MergeTableRegionsState.valueOf(stateId);
+    return MergeTableRegionsState.forNumber(stateId);
   }
 
   @Override
@@ -613,11 +613,8 @@ public class MergeTableRegionsProcedure
         final CacheConfig cacheConf = new CacheConfig(conf, hcd);
         for (StoreFileInfo storeFileInfo: storeFiles) {
           // Create reference file(s) of the region in mergedDir
-          regionFs.mergeStoreFile(
-            mergedRegionInfo,
-            family,
-            new StoreFile(
-              mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()),
+          regionFs.mergeStoreFile(mergedRegionInfo, family, new StoreFile(mfs.getFileSystem(),
+              storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true),
             mergedDir);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
index 3cd6c66..bf9afd7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
@@ -285,7 +285,7 @@ public class SplitTableRegionProcedure
 
   @Override
   protected SplitTableRegionState getState(final int stateId) {
-    return SplitTableRegionState.valueOf(stateId);
+    return SplitTableRegionState.forNumber(stateId);
   }
 
   @Override
@@ -571,9 +571,9 @@ public class SplitTableRegionProcedure
       if (storeFiles != null && storeFiles.size() > 0) {
         final CacheConfig cacheConf = new CacheConfig(conf, hcd);
         for (StoreFileInfo storeFileInfo: storeFiles) {
-          StoreFileSplitter sfs = new StoreFileSplitter(regionFs, family.getBytes(),
-            new StoreFile(mfs.getFileSystem(), storeFileInfo, conf,
-              cacheConf, hcd.getBloomFilterType()));
+          StoreFileSplitter sfs =
+              new StoreFileSplitter(regionFs, family.getBytes(), new StoreFile(mfs.getFileSystem(),
+                  storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true));
           futures.add(threadPool.submit(sfs));
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
index 7c4d6fe..90d1f2d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
@@ -44,7 +44,9 @@ public class CachedMobFile extends MobFile implements Comparable<CachedMobFile>
 
   public static CachedMobFile create(FileSystem fs, Path path, Configuration conf,
       CacheConfig cacheConf) throws IOException {
-    StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE);
+    // XXX: primaryReplica is only used for constructing the key of block cache so it is not a
+    // critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
+    StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
     return new CachedMobFile(sf);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
index cd4c079..73355e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
@@ -118,9 +118,7 @@ public class MobFile {
    * @throws IOException
    */
   public void open() throws IOException {
-    if (sf.getReader() == null) {
-      sf.createReader();
-    }
+    sf.initReader();
   }
 
   /**
@@ -146,7 +144,9 @@ public class MobFile {
    */
   public static MobFile create(FileSystem fs, Path path, Configuration conf, CacheConfig cacheConf)
       throws IOException {
-    StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE);
+    // XXX: primaryReplica is only used for constructing the key of block cache so it is not a
+    // critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
+    StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
     return new MobFile(sf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index eb75120..06c5001 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -333,7 +333,8 @@ public final class MobUtils {
           if (LOG.isDebugEnabled()) {
             LOG.debug(fileName + " is an expired file");
           }
-          filesToClean.add(new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE));
+          filesToClean
+              .add(new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true));
         }
       } catch (Exception e) {
         LOG.error("Cannot parse the fileName " + fileName, e);
@@ -372,7 +373,7 @@ public final class MobUtils {
     Path hbaseDir = new Path(conf.get(HConstants.HBASE_DIR));
     Path mobRootDir = new Path(hbaseDir, MobConstants.MOB_DIR_NAME);
     FileSystem fs = mobRootDir.getFileSystem(conf);
-    return mobRootDir.makeQualified(fs);
+    return mobRootDir.makeQualified(fs.getUri(), fs.getWorkingDirectory());
   }
 
   /**
@@ -697,7 +698,7 @@ public final class MobUtils {
       return null;
     }
     Path dstPath = new Path(targetPath, sourceFile.getName());
-    validateMobFile(conf, fs, sourceFile, cacheConfig);
+    validateMobFile(conf, fs, sourceFile, cacheConfig, true);
     String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
     LOG.info(msg);
     Path parent = dstPath.getParent();
@@ -718,11 +719,11 @@ public final class MobUtils {
    * @param cacheConfig The current cache config.
    */
   private static void validateMobFile(Configuration conf, FileSystem fs, Path path,
-      CacheConfig cacheConfig) throws IOException {
+      CacheConfig cacheConfig, boolean primaryReplica) throws IOException {
     StoreFile storeFile = null;
     try {
-      storeFile = new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE);
-      storeFile.createReader();
+      storeFile = new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
+      storeFile.initReader();
     } catch (IOException e) {
       LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e);
       throw e;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index 987fe51..05c7076 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -223,12 +223,9 @@ public class PartitionedMobCompactor extends MobCompactor {
         // File in the Del Partition List
 
         // Get delId from the file
-        Reader reader = HFile.createReader(fs, linkedFile.getPath(), CacheConfig.DISABLED, conf);
-        try {
+        try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
           delId.setStartKey(reader.getFirstRowKey());
           delId.setEndKey(reader.getLastRowKey());
-        } finally {
-          reader.close();
         }
         CompactionDelPartition delPartition = delFilesToCompact.get(delId);
         if (delPartition == null) {
@@ -267,12 +264,9 @@ public class PartitionedMobCompactor extends MobCompactor {
           if (withDelFiles) {
             // get startKey and endKey from the file and update partition
             // TODO: is it possible to skip read of most hfiles?
-            Reader reader = HFile.createReader(fs, linkedFile.getPath(), CacheConfig.DISABLED, conf);
-            try {
+            try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
               compactionPartition.setStartKey(reader.getFirstRowKey());
               compactionPartition.setEndKey(reader.getLastRowKey());
-            } finally {
-              reader.close();
             }
           }
 
@@ -340,10 +334,11 @@ public class PartitionedMobCompactor extends MobCompactor {
     try {
       for (CompactionDelPartition delPartition : request.getDelPartitions()) {
         for (Path newDelPath : delPartition.listDelFiles()) {
-          StoreFile sf = new StoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE);
+          StoreFile sf =
+              new StoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true);
           // pre-create reader of a del file to avoid race condition when opening the reader in each
           // partition.
-          sf.createReader();
+          sf.initReader();
           delPartition.addStoreFile(sf);
           totalDelFileCount++;
         }
@@ -557,7 +552,7 @@ public class PartitionedMobCompactor extends MobCompactor {
       List<StoreFile> filesToCompact = new ArrayList<>();
       for (int i = offset; i < batch + offset; i++) {
         StoreFile sf = new StoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
-          BloomType.NONE);
+            BloomType.NONE, true);
         filesToCompact.add(sf);
       }
       filesToCompact.addAll(delFiles);
@@ -739,7 +734,7 @@ public class PartitionedMobCompactor extends MobCompactor {
       }
       for (int i = offset; i < batch + offset; i++) {
         batchedDelFiles.add(new StoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
-          BloomType.NONE));
+          BloomType.NONE, true));
       }
       // compact the del files in a batch.
       paths.add(compactDelFilesInBatch(request, batchedDelFiles));
@@ -809,8 +804,8 @@ public class PartitionedMobCompactor extends MobCompactor {
    */
   private StoreScanner createScanner(List<StoreFile> filesToCompact, ScanType scanType)
     throws IOException {
-    List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, true, false,
-      false, HConstants.LATEST_TIMESTAMP);
+    List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact,
+      false, true, false, false, HConstants.LATEST_TIMESTAMP);
     Scan scan = new Scan();
     scan.setMaxVersions(column.getMaxVersions());
     long ttl = HStore.determineTTLFromFamily(column);
@@ -893,7 +888,8 @@ public class PartitionedMobCompactor extends MobCompactor {
     for (StoreFile sf : storeFiles) {
       // the readers will be closed later after the merge.
       maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId());
-      byte[] count = sf.createReader().loadFileInfo().get(StoreFile.MOB_CELLS_COUNT);
+      sf.initReader();
+      byte[] count = sf.getReader().loadFileInfo().get(StoreFile.MOB_CELLS_COUNT);
       if (count != null) {
         maxKeyCount += Bytes.toLong(count);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
index c37ae99..da25df5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
@@ -135,7 +135,7 @@ class DefaultStoreFileManager implements StoreFileManager {
     this.compactedfiles = sortCompactedfiles(updatedCompactedfiles);
   }
 
-  // Mark the files as compactedAway once the storefiles and compactedfiles list is finalised
+  // Mark the files as compactedAway once the storefiles and compactedfiles list is finalized
   // Let a background thread close the actual reader on these compacted files and also
   // ensure to evict the blocks from block cache so that they are no longer in
   // cache

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index b021430..032e383 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -292,9 +292,9 @@ public class HMobStore extends HStore {
   private void validateMobFile(Path path) throws IOException {
     StoreFile storeFile = null;
     try {
-      storeFile =
-          new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, BloomType.NONE);
-      storeFile.createReader();
+      storeFile = new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
+          BloomType.NONE, isPrimaryReplicaStore());
+      storeFile.initReader();
     } catch (IOException e) {
       LOG.error("Fail to open mob file[" + path + "], keep it in temp directory.", e);
       throw e;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/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 78ce608..b21a84d 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
@@ -4160,8 +4160,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regiondir).getName();
       Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size());
       for (Path file: files) {
-        fakeStoreFiles.add(new StoreFile(getRegionFileSystem().getFileSystem(), file, this.conf,
-          null, null));
+        fakeStoreFiles.add(
+          new StoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true));
       }
       getRegionFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 144f43b..014427d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -294,7 +294,7 @@ public class HRegionFileSystem {
    */
   Path getStoreFilePath(final String familyName, final String fileName) {
     Path familyDir = getStoreDir(familyName);
-    return new Path(familyDir, fileName).makeQualified(this.fs);
+    return new Path(familyDir, fileName).makeQualified(fs.getUri(), fs.getWorkingDirectory());
   }
 
   /**
@@ -675,9 +675,7 @@ public class HRegionFileSystem {
     if (splitPolicy == null || !splitPolicy.skipStoreFileRangeCheck(familyName)) {
       // Check whether the split row lies in the range of the store file
       // If it is outside the range, return directly.
-      if (f.getReader() == null) {
-        f.createReader();
-      }
+      f.initReader();
       try {
         if (top) {
           //check if larger than last key.

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index a98f89e..cbdaa1b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -650,13 +650,11 @@ public class HStore implements Store {
     return createStoreFileAndReader(info);
   }
 
-  private StoreFile createStoreFileAndReader(final StoreFileInfo info)
-      throws IOException {
+  private StoreFile createStoreFileAndReader(final StoreFileInfo info) throws IOException {
     info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
     StoreFile storeFile = new StoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
-      this.family.getBloomFilterType());
-    StoreFileReader r = storeFile.createReader();
-    r.setReplicaStoreFile(isPrimaryReplicaStore());
+        this.family.getBloomFilterType(), isPrimaryReplicaStore());
+    storeFile.initReader();
     return storeFile;
   }
 
@@ -726,8 +724,8 @@ public class HStore implements Store {
     try {
       LOG.info("Validating hfile at " + srcPath + " for inclusion in "
           + "store " + this + " region " + this.getRegionInfo().getRegionNameAsString());
-      reader = HFile.createReader(srcPath.getFileSystem(conf),
-          srcPath, cacheConf, conf);
+      reader = HFile.createReader(srcPath.getFileSystem(conf), srcPath, cacheConf,
+        isPrimaryReplicaStore(), conf);
       reader.loadFileInfo();
 
       byte[] firstKey = reader.getFirstRowKey();
@@ -1180,7 +1178,7 @@ public class HStore implements Store {
     // but now we get them in ascending order, which I think is
     // actually more correct, since memstore get put at the end.
     List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(storeFilesToScan,
-      cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
+      cacheBlocks, usePread, isCompaction, false, matcher, readPt);
     List<KeyValueScanner> scanners = new ArrayList<>(sfScanners.size() + 1);
     scanners.addAll(sfScanners);
     // Then the memstore scanners
@@ -1203,7 +1201,7 @@ public class HStore implements Store {
       }
     }
     List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(files,
-      cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
+      cacheBlocks, usePread, isCompaction, false, matcher, readPt);
     List<KeyValueScanner> scanners = new ArrayList<>(sfScanners.size() + 1);
     scanners.addAll(sfScanners);
     // Then the memstore scanners
@@ -2456,8 +2454,9 @@ public class HStore implements Store {
               LOG.debug("The file " + file + " was closed but still not archived.");
             }
             filesToRemove.add(file);
+            continue;
           }
-          if (r != null && r.isCompactedAway() && !r.isReferencedInReads()) {
+          if (file.isCompactedAway() && !file.isReferencedInReads()) {
             // Even if deleting fails we need not bother as any new scanners won't be
             // able to use the compacted file as the status is already compactedAway
             if (LOG.isTraceEnabled()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
index 41c13f5..d71af2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
@@ -54,7 +54,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
 
   /** Constructor for testing. */
   ReversedStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType,
-      final NavigableSet<byte[]> columns, final List<KeyValueScanner> scanners)
+      final NavigableSet<byte[]> columns, final List<? extends KeyValueScanner> scanners)
       throws IOException {
     super(scan, scanInfo, scanType, columns, scanners,
         HConstants.LATEST_TIMESTAMP);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 7aef05e..c53fbf08 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -30,6 +30,7 @@ import java.util.Comparator;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -54,7 +55,7 @@ import org.apache.hadoop.hbase.util.Bytes;
  * and append data. Be sure to add any metadata before calling close on the
  * Writer (Use the appendMetadata convenience methods). On close, a StoreFile
  * is sitting in the Filesystem.  To refer to it, create a StoreFile instance
- * passing filesystem and path.  To read, call {@link #createReader()}.
+ * passing filesystem and path.  To read, call {@link #initReader()}
  * <p>StoreFiles may also reference store files in another Store.
  *
  * The reason for this weird pattern where you use a different instance for the
@@ -64,6 +65,10 @@ import org.apache.hadoop.hbase.util.Bytes;
 public class StoreFile {
   private static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
 
+  public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
+
+  private static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
+
   // Keys for fileinfo values in HFile
 
   /** Max Sequence ID in FileInfo */
@@ -103,6 +108,18 @@ public class StoreFile {
   // Block cache configuration and reference.
   private final CacheConfig cacheConf;
 
+  // Counter that is incremented every time a scanner is created on the
+  // store file. It is decremented when the scan on the store file is
+  // done.
+  private final AtomicInteger refCount = new AtomicInteger(0);
+
+  private final boolean noReadahead;
+
+  private final boolean primaryReplica;
+
+  // Indicates if the file got compacted
+  private volatile boolean compactedAway = false;
+
   // Keys for metadata stored in backing HFile.
   // Set when we obtain a Reader.
   private long sequenceid = -1;
@@ -116,7 +133,7 @@ public class StoreFile {
 
   private Cell lastKey;
 
-  private Comparator comparator;
+  private Comparator<Cell> comparator;
 
   CacheConfig getCacheConf() {
     return cacheConf;
@@ -130,7 +147,7 @@ public class StoreFile {
     return lastKey;
   }
 
-  public Comparator getComparator() {
+  public Comparator<Cell> getComparator() {
     return comparator;
   }
 
@@ -179,72 +196,96 @@ public class StoreFile {
   public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
 
   /**
-   * Constructor, loads a reader and it's indices, etc. May allocate a
-   * substantial amount of ram depending on the underlying files (10-20MB?).
-   *
-   * @param fs  The current file system to use.
-   * @param p  The path of the file.
-   * @param conf  The current configuration.
-   * @param cacheConf  The cache configuration and block cache reference.
-   * @param cfBloomType The bloom type to use for this store file as specified
-   *          by column family configuration. This may or may not be the same
-   *          as the Bloom filter type actually present in the HFile, because
-   *          column family configuration might change. If this is
+   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
+   * depending on the underlying files (10-20MB?).
+   * @param fs The current file system to use.
+   * @param p The path of the file.
+   * @param conf The current configuration.
+   * @param cacheConf The cache configuration and block cache reference.
+   * @param cfBloomType The bloom type to use for this store file as specified by column family
+   *          configuration. This may or may not be the same as the Bloom filter type actually
+   *          present in the HFile, because column family configuration might change. If this is
    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
-   * @throws IOException When opening the reader fails.
+   * @deprecated Now we will specific whether the StoreFile is for primary replica when
+   *             constructing, so please use
+   *             {@link #StoreFile(FileSystem, Path, Configuration, CacheConfig, BloomType, boolean)}
+   *             directly.
    */
+  @Deprecated
   public StoreFile(final FileSystem fs, final Path p, final Configuration conf,
-        final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
+      final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
     this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
   }
 
   /**
-   * Constructor, loads a reader and it's indices, etc. May allocate a
-   * substantial amount of ram depending on the underlying files (10-20MB?).
-   *
-   * @param fs  The current file system to use.
-   * @param fileInfo  The store file information.
-   * @param conf  The current configuration.
-   * @param cacheConf  The cache configuration and block cache reference.
-   * @param cfBloomType The bloom type to use for this store file as specified
-   *          by column family configuration. This may or may not be the same
-   *          as the Bloom filter type actually present in the HFile, because
-   *          column family configuration might change. If this is
+   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
+   * depending on the underlying files (10-20MB?).
+   * @param fs The current file system to use.
+   * @param p The path of the file.
+   * @param conf The current configuration.
+   * @param cacheConf The cache configuration and block cache reference.
+   * @param cfBloomType The bloom type to use for this store file as specified by column family
+   *          configuration. This may or may not be the same as the Bloom filter type actually
+   *          present in the HFile, because column family configuration might change. If this is
+   *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
+   * @param primaryReplica true if this is a store file for primary replica, otherwise false.
+   * @throws IOException
+   */
+  public StoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheConf,
+      BloomType cfBloomType, boolean primaryReplica) throws IOException {
+    this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType, primaryReplica);
+  }
+
+  /**
+   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
+   * depending on the underlying files (10-20MB?).
+   * @param fs The current file system to use.
+   * @param fileInfo The store file information.
+   * @param conf The current configuration.
+   * @param cacheConf The cache configuration and block cache reference.
+   * @param cfBloomType The bloom type to use for this store file as specified by column family
+   *          configuration. This may or may not be the same as the Bloom filter type actually
+   *          present in the HFile, because column family configuration might change. If this is
    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
-   * @throws IOException When opening the reader fails.
+   * @deprecated Now we will specific whether the StoreFile is for primary replica when
+   *             constructing, so please use
+   *             {@link #StoreFile(FileSystem, StoreFileInfo, Configuration, CacheConfig, BloomType, boolean)}
+   *             directly.
    */
+  @Deprecated
   public StoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
-      final CacheConfig cacheConf,  final BloomType cfBloomType) throws IOException {
+      final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
+    this(fs, fileInfo, conf, cacheConf, cfBloomType, true);
+  }
+
+  /**
+   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
+   * depending on the underlying files (10-20MB?).
+   * @param fs fs The current file system to use.
+   * @param fileInfo The store file information.
+   * @param conf The current configuration.
+   * @param cacheConf The cache configuration and block cache reference.
+   * @param cfBloomType cfBloomType The bloom type to use for this store file as specified by column
+   *          family configuration. This may or may not be the same as the Bloom filter type
+   *          actually present in the HFile, because column family configuration might change. If
+   *          this is {@link BloomType#NONE}, the existing Bloom filter is ignored.
+   * @param primaryReplica true if this is a store file for primary replica, otherwise false.
+   */
+  public StoreFile(FileSystem fs, StoreFileInfo fileInfo, Configuration conf, CacheConfig cacheConf,
+      BloomType cfBloomType, boolean primaryReplica) {
     this.fs = fs;
     this.fileInfo = fileInfo;
     this.cacheConf = cacheConf;
-
+    this.noReadahead =
+        conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
     if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
       this.cfBloomType = cfBloomType;
     } else {
-      LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " +
-          "cfBloomType=" + cfBloomType + " (disabled in config)");
+      LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " + "cfBloomType=" +
+          cfBloomType + " (disabled in config)");
       this.cfBloomType = BloomType.NONE;
     }
-  }
-
-  /**
-   * Clone
-   * @param other The StoreFile to clone from
-   */
-  public StoreFile(final StoreFile other) {
-    this.fs = other.fs;
-    this.fileInfo = other.fileInfo;
-    this.cacheConf = other.cacheConf;
-    this.cfBloomType = other.cfBloomType;
-    this.metadataMap = other.metadataMap;
-  }
-
-  /**
-   * Clone a StoreFile for opening private reader.
-   */
-  public StoreFile cloneForReader() {
-    return new StoreFile(this);
+    this.primaryReplica = primaryReplica;
   }
 
   /**
@@ -266,12 +307,12 @@ public class StoreFile {
    * @return Returns the qualified path of this StoreFile
    */
   public Path getQualifiedPath() {
-    return this.fileInfo.getPath().makeQualified(fs);
+    return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory());
   }
 
   /**
    * @return True if this is a StoreFile Reference; call
-   * after {@link #open(boolean canUseDropBehind)} else may get wrong answer.
+   * after {@link #open()} else may get wrong answer.
    */
   public boolean isReference() {
     return this.fileInfo.isReference();
@@ -376,15 +417,21 @@ public class StoreFile {
 
   @VisibleForTesting
   public boolean isCompactedAway() {
-    if (this.reader != null) {
-      return this.reader.isCompactedAway();
-    }
-    return true;
+    return compactedAway;
   }
 
   @VisibleForTesting
   public int getRefCount() {
-    return this.reader.getRefCount().get();
+    return refCount.get();
+  }
+
+  /**
+   * @return true if the file is still used in reads
+   */
+  public boolean isReferencedInReads() {
+    int rc = refCount.get();
+    assert rc >= 0; // we should not go negative.
+    return rc > 0;
   }
 
   /**
@@ -404,18 +451,18 @@ public class StoreFile {
   }
 
   /**
-   * Opens reader on this store file.  Called by Constructor.
-   * @return Reader for the store file.
+   * Opens reader on this store file. Called by Constructor.
    * @throws IOException
    * @see #closeReader(boolean)
    */
-  private StoreFileReader open(boolean canUseDropBehind) throws IOException {
+  private void open() throws IOException {
     if (this.reader != null) {
       throw new IllegalAccessError("Already open");
     }
 
     // Open the StoreFile.Reader
-    this.reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind);
+    this.reader = fileInfo.open(this.fs, this.cacheConf, false, noReadahead ? 0L : -1L,
+      primaryReplica, refCount, true);
 
     // Load up indices and fileinfo. This also loads Bloom filter type.
     metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
@@ -513,38 +560,45 @@ public class StoreFile {
     firstKey = reader.getFirstKey();
     lastKey = reader.getLastKey();
     comparator = reader.getComparator();
-    return this.reader;
-  }
-
-  public StoreFileReader createReader() throws IOException {
-    return createReader(false);
   }
 
   /**
-   * @return Reader for StoreFile. creates if necessary
-   * @throws IOException
+   * Initialize the reader used for pread.
    */
-  public StoreFileReader createReader(boolean canUseDropBehind) throws IOException {
-    if (this.reader == null) {
+  public void initReader() throws IOException {
+    if (reader == null) {
       try {
-        this.reader = open(canUseDropBehind);
-      } catch (IOException e) {
+        open();
+      } catch (Exception e) {
         try {
-          boolean evictOnClose =
-              cacheConf != null? cacheConf.shouldEvictOnClose(): true;
+          boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
           this.closeReader(evictOnClose);
         } catch (IOException ee) {
+          LOG.warn("failed to close reader", ee);
         }
         throw e;
       }
-
     }
-    return this.reader;
+  }
+
+  private StoreFileReader createStreamReader(boolean canUseDropBehind) throws IOException {
+    initReader();
+    StoreFileReader reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind, -1L,
+      primaryReplica, refCount, false);
+    reader.copyFields(this.reader);
+    return reader;
+  }
+
+  public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
+      boolean pread, boolean isCompaction, long readPt, long scannerOrder,
+      boolean canOptimizeForNonNullColumn) throws IOException {
+    return createStreamReader(canUseDropBehind).getStoreFileScanner(
+      cacheBlocks, pread, isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn);
   }
 
   /**
-   * @return Current reader.  Must call createReader first else returns null.
-   * @see #createReader()
+   * @return Current reader.  Must call initReader first else returns null.
+   * @see #initReader()
    */
   public StoreFileReader getReader() {
     return this.reader;
@@ -566,9 +620,7 @@ public class StoreFile {
    * Marks the status of the file as compactedAway.
    */
   public void markCompactedAway() {
-    if (this.reader != null) {
-      this.reader.markCompactedAway();
-    }
+    this.compactedAway = true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
index 3c12045..c4754a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
@@ -21,17 +21,18 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Matcher;
 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.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
@@ -233,25 +234,24 @@ public class StoreFileInfo {
    * @param cacheConf The cache configuration and block cache reference.
    * @return The StoreFile.Reader for the file
    */
-  public StoreFileReader open(final FileSystem fs,
-      final CacheConfig cacheConf, final boolean canUseDropBehind) throws IOException {
+  public StoreFileReader open(FileSystem fs, CacheConfig cacheConf, boolean canUseDropBehind,
+      long readahead, boolean isPrimaryReplicaStoreFile, AtomicInteger refCount, boolean shared)
+      throws IOException {
     FSDataInputStreamWrapper in;
     FileStatus status;
 
     final boolean doDropBehind = canUseDropBehind && cacheConf.shouldDropBehindCompaction();
     if (this.link != null) {
       // HFileLink
-      in = new FSDataInputStreamWrapper(fs, this.link, doDropBehind);
+      in = new FSDataInputStreamWrapper(fs, this.link, doDropBehind, readahead);
       status = this.link.getFileStatus(fs);
     } else if (this.reference != null) {
       // HFile Reference
       Path referencePath = getReferredToFile(this.getPath());
-      in = new FSDataInputStreamWrapper(fs, referencePath,
-          doDropBehind);
+      in = new FSDataInputStreamWrapper(fs, referencePath, doDropBehind, readahead);
       status = fs.getFileStatus(referencePath);
     } else {
-      in = new FSDataInputStreamWrapper(fs, this.getPath(),
-          doDropBehind);
+      in = new FSDataInputStreamWrapper(fs, this.getPath(), doDropBehind, readahead);
       status = fs.getFileStatus(initialPath);
     }
     long length = status.getLen();
@@ -265,9 +265,10 @@ public class StoreFileInfo {
     if (reader == null) {
       if (this.reference != null) {
         reader = new HalfStoreFileReader(fs, this.getPath(), in, length, cacheConf, reference,
-          conf);
+            isPrimaryReplicaStoreFile, refCount, shared, conf);
       } else {
-        reader = new StoreFileReader(fs, status.getPath(), in, length, cacheConf, conf);
+        reader = new StoreFileReader(fs, status.getPath(), in, length, cacheConf,
+            isPrimaryReplicaStoreFile, refCount, shared, conf);
       }
     }
     if (this.coprocessorHost != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
index 8f01a93..b015ea5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.Map;
@@ -34,7 +36,6 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 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.client.Scan;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
@@ -68,36 +69,47 @@ public class StoreFileReader {
   private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
   private boolean skipResetSeqId = true;
 
-  public AtomicInteger getRefCount() {
-    return refCount;
-  }
-
   // Counter that is incremented every time a scanner is created on the
-  // store file.  It is decremented when the scan on the store file is
-  // done.
-  private AtomicInteger refCount = new AtomicInteger(0);
-  // Indicates if the file got compacted
-  private volatile boolean compactedAway = false;
+  // store file. It is decremented when the scan on the store file is
+  // done. All StoreFileReader for the same StoreFile will share this counter.
+  private final AtomicInteger refCount;
 
-  public StoreFileReader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
-      throws IOException {
-    reader = HFile.createReader(fs, path, cacheConf, conf);
+  // indicate that whether this StoreFileReader is shared, i.e., used for pread. If not, we will
+  // close the internal reader when readCompleted is called.
+  private final boolean shared;
+
+  private StoreFileReader(HFile.Reader reader, AtomicInteger refCount, boolean shared) {
+    this.reader = reader;
     bloomFilterType = BloomType.NONE;
+    this.refCount = refCount;
+    this.shared = shared;
   }
 
-  void markCompactedAway() {
-    this.compactedAway = true;
+  public StoreFileReader(FileSystem fs, Path path, CacheConfig cacheConf,
+      boolean primaryReplicaStoreFile, AtomicInteger refCount, boolean shared, Configuration conf)
+      throws IOException {
+    this(HFile.createReader(fs, path, cacheConf, primaryReplicaStoreFile, conf), refCount, shared);
   }
 
   public StoreFileReader(FileSystem fs, Path path, FSDataInputStreamWrapper in, long size,
-      CacheConfig cacheConf, Configuration conf) throws IOException {
-    reader = HFile.createReader(fs, path, in, size, cacheConf, conf);
-    bloomFilterType = BloomType.NONE;
+      CacheConfig cacheConf, boolean primaryReplicaStoreFile, AtomicInteger refCount,
+      boolean shared, Configuration conf) throws IOException {
+    this(HFile.createReader(fs, path, in, size, cacheConf, primaryReplicaStoreFile, conf), refCount,
+        shared);
   }
 
-  public void setReplicaStoreFile(boolean isPrimaryReplicaStoreFile) {
-    reader.setPrimaryReplicaReader(isPrimaryReplicaStoreFile);
+  void copyFields(StoreFileReader reader) {
+    this.generalBloomFilter = reader.generalBloomFilter;
+    this.deleteFamilyBloomFilter = reader.deleteFamilyBloomFilter;
+    this.bloomFilterType = reader.bloomFilterType;
+    this.sequenceID = reader.sequenceID;
+    this.timeRange = reader.timeRange;
+    this.lastBloomKey = reader.lastBloomKey;
+    this.bulkLoadResult = reader.bulkLoadResult;
+    this.lastBloomKeyOnlyKV = reader.lastBloomKeyOnlyKV;
+    this.skipResetSeqId = reader.skipResetSeqId;
   }
+
   public boolean isPrimaryReplicaReader() {
     return reader.isPrimaryReplicaReader();
   }
@@ -105,8 +117,11 @@ public class StoreFileReader {
   /**
    * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
    */
+  @VisibleForTesting
   StoreFileReader() {
+    this.refCount = new AtomicInteger(0);
     this.reader = null;
+    this.shared = false;
   }
 
   public CellComparator getComparator() {
@@ -128,30 +143,23 @@ public class StoreFileReader {
       boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) {
     // Increment the ref count
     refCount.incrementAndGet();
-    return new StoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction), !isCompaction,
-        reader.hasMVCCInfo(), readPt, scannerOrder, canOptimizeForNonNullColumn);
+    return new StoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction),
+        !isCompaction, reader.hasMVCCInfo(), readPt, scannerOrder, canOptimizeForNonNullColumn);
   }
 
   /**
-   * Decrement the ref count associated with the reader when ever a scanner associated
-   * with the reader is closed
+   * Indicate that the scanner has finished reading with this reader. We need to decrement the ref
+   * count, and also, if this is not the common pread reader, we should close it.
    */
-  void decrementRefCount() {
+  void readCompleted() {
     refCount.decrementAndGet();
-  }
-
-  /**
-   * @return true if the file is still used in reads
-   */
-  public boolean isReferencedInReads() {
-    return refCount.get() != 0;
-  }
-
-  /**
-   * @return true if the file is compacted
-   */
-  public boolean isCompactedAway() {
-    return this.compactedAway;
+    if (!shared) {
+      try {
+        reader.close(false);
+      } catch (IOException e) {
+        LOG.warn("failed to close stream reader", e);
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
index ab6b0ef..aa4f897 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
@@ -31,8 +31,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 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.client.Scan;
 import org.apache.hadoop.hbase.io.TimeRange;
@@ -124,26 +122,44 @@ public class StoreFileScanner implements KeyValueScanner {
    */
   public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
       boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
-      ScanQueryMatcher matcher, long readPt, boolean isPrimaryReplica) throws IOException {
+      ScanQueryMatcher matcher, long readPt) throws IOException {
     List<StoreFileScanner> scanners = new ArrayList<>(files.size());
-    List<StoreFile> sorted_files = new ArrayList<>(files);
-    Collections.sort(sorted_files, StoreFile.Comparators.SEQ_ID);
-    for (int i = 0; i < sorted_files.size(); i++) {
-      StoreFileReader r = sorted_files.get(i).createReader(canUseDrop);
-      r.setReplicaStoreFile(isPrimaryReplica);
-      StoreFileScanner scanner = r.getStoreFileScanner(cacheBlocks, usePread, isCompaction, readPt,
-        i, matcher != null ? !matcher.hasNullColumnInQuery() : false);
+    List<StoreFile> sortedFiles = new ArrayList<>(files);
+    Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID);
+    for (int i = 0, n = sortedFiles.size(); i < n; i++) {
+      StoreFile sf = sortedFiles.get(i);
+      sf.initReader();
+      StoreFileScanner scanner = sf.getReader().getStoreFileScanner(cacheBlocks, usePread,
+        isCompaction, readPt, i, matcher != null ? !matcher.hasNullColumnInQuery() : false);
       scanners.add(scanner);
     }
     return scanners;
   }
 
-  public static List<StoreFileScanner> getScannersForStoreFiles(
-    Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
-    boolean isCompaction, boolean canUseDrop,
-    ScanQueryMatcher matcher, long readPt) throws IOException {
-    return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction, canUseDrop,
-      matcher, readPt, true);
+  /**
+   * Get scanners for compaction. We will create a separated reader for each store file to avoid
+   * contention with normal read request.
+   */
+  public static List<StoreFileScanner> getScannersForCompaction(Collection<StoreFile> files,
+      boolean canUseDropBehind, long readPt) throws IOException {
+    List<StoreFileScanner> scanners = new ArrayList<>(files.size());
+    List<StoreFile> sortedFiles = new ArrayList<>(files);
+    Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID);
+    boolean succ = false;
+    try {
+      for (int i = 0, n = sortedFiles.size(); i < n; i++) {
+        scanners.add(sortedFiles.get(i).getStreamScanner(canUseDropBehind, false, false, true,
+          readPt, i, false));
+      }
+      succ = true;
+    } finally {
+      if (!succ) {
+        for (StoreFileScanner scanner : scanners) {
+          scanner.close();
+        }
+      }
+    }
+    return scanners;
   }
 
   public String toString() {
@@ -262,7 +278,7 @@ public class StoreFileScanner implements KeyValueScanner {
     cur = null;
     this.hfs.close();
     if (this.reader != null) {
-      this.reader.decrementRefCount();
+      this.reader.readCompleted();
     }
     closed = true;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 99ec30e..3bc6a0f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -312,7 +312,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   @VisibleForTesting
   StoreScanner(final Scan scan, ScanInfo scanInfo,
       ScanType scanType, final NavigableSet<byte[]> columns,
-      final List<KeyValueScanner> scanners) throws IOException {
+      final List<? extends KeyValueScanner> scanners) throws IOException {
     this(scan, scanInfo, scanType, columns, scanners,
         HConstants.LATEST_TIMESTAMP,
         // 0 is passed as readpoint because the test bypasses Store
@@ -322,7 +322,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   @VisibleForTesting
   StoreScanner(final Scan scan, ScanInfo scanInfo,
     ScanType scanType, final NavigableSet<byte[]> columns,
-    final List<KeyValueScanner> scanners, long earliestPutTs)
+    final List<? extends KeyValueScanner> scanners, long earliestPutTs)
         throws IOException {
     this(scan, scanInfo, scanType, columns, scanners, earliestPutTs,
       // 0 is passed as readpoint because the test bypasses Store
@@ -330,7 +330,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   }
 
   public StoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType,
-      final NavigableSet<byte[]> columns, final List<KeyValueScanner> scanners, long earliestPutTs,
+      final NavigableSet<byte[]> columns, final List<? extends KeyValueScanner> scanners, long earliestPutTs,
       long readPt) throws IOException {
     this(null, scan, scanInfo, columns, readPt,
         scanType == ScanType.USER_SCAN ? scan.getCacheBlocks() : false);


[06/40] hbase git commit: HBASE-16215 clean up of ref guide and site for EOM versions.

Posted by sy...@apache.org.
HBASE-16215 clean up of ref guide and site for EOM versions.

Signed-off-by: Enis Soztutar <en...@apache.org>
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/a8e6f337
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a8e6f337
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a8e6f337

Branch: refs/heads/hbase-12439
Commit: a8e6f33791d787eaf9c1bfe63aa7f3266f25268b
Parents: d15f75b
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Apr 12 09:04:44 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Apr 18 16:52:30 2017 -0500

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/community.adoc     |   9 +-
 src/main/asciidoc/_chapters/configuration.adoc | 153 +++-----------------
 src/main/asciidoc/_chapters/cp.adoc            |  10 --
 src/main/asciidoc/_chapters/developer.adoc     |  60 ++------
 src/main/asciidoc/_chapters/upgrading.adoc     |   6 +-
 src/main/site/site.xml                         |   5 -
 6 files changed, 42 insertions(+), 201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a8e6f337/src/main/asciidoc/_chapters/community.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/community.adoc b/src/main/asciidoc/_chapters/community.adoc
index ba07df7..f63d597 100644
--- a/src/main/asciidoc/_chapters/community.adoc
+++ b/src/main/asciidoc/_chapters/community.adoc
@@ -62,12 +62,11 @@ Any -1 on a patch by anyone vetoes a patch; it cannot be committed until the jus
 .How to set fix version in JIRA on issue resolve
 
 Here is how link:http://search-hadoop.com/m/azemIi5RCJ1[we agreed] to set versions in JIRA when we resolve an issue.
-If master is going to be 0.98.0 then:
+If master is going to be 2.0.0, and branch-1 1.4.0 then:
 
-* Commit only to master: Mark with 0.98
-* Commit to 0.95 and master: Mark with 0.98, and 0.95.x
-* Commit to 0.94.x and 0.95, and master: Mark with 0.98, 0.95.x, and 0.94.x
-* Commit to 89-fb: Mark with 89-fb.
+* Commit only to master: Mark with 2.0.0
+* Commit to branch-1 and master: Mark with 2.0.0, and 1.4.0
+* Commit to branch-1.3, branch-1, and master: Mark with 2.0.0, 1.4.0, and 1.3.x
 * Commit site fixes: no version
 
 [[hbase.when.to.close.jira]]

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8e6f337/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index b6b6c15..ff4bf6a 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -93,54 +93,34 @@ This section lists required services and some required system configuration.
 
 [[java]]
 .Java
-[cols="1,1,1,4", options="header"]
+[cols="1,1,4", options="header"]
 |===
 |HBase Version
-|JDK 6
 |JDK 7
 |JDK 8
 
 |2.0
-|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
 |link:http://search-hadoop.com/m/YGbbsPxZ723m3as[Not Supported]
 |yes
 
 |1.3
-|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
 |yes
 |yes
 
 
 |1.2
-|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
 |yes
 |yes
 
 |1.1
-|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
 |yes
 |Running with JDK 8 will work but is not well tested.
 
-|1.0
-|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
-|yes
-|Running with JDK 8 will work but is not well tested.
-
-|0.98
-|yes
-|yes
-|Running with JDK 8 works but is not well tested. Building with JDK 8 would require removal of the
-deprecated `remove()` method of the `PoolMap` class and is under consideration. See
-link:https://issues.apache.org/jira/browse/HBASE-7608[HBASE-7608] for more information about JDK 8
-support.
-
-|0.94
-|yes
-|yes
-|N/A
 |===
 
-NOTE: In HBase 0.98.5 and newer, you must set `JAVA_HOME` on each node of your cluster. _hbase-env.sh_ provides a handy mechanism to do this.
+NOTE: HBase will neither build nor compile with Java 6.
+
+NOTE: You must set `JAVA_HOME` on each node of your cluster. _hbase-env.sh_ provides a handy mechanism to do this.
 
 [[os]]
 .Operating System Utilities
@@ -213,8 +193,8 @@ See link:http://wiki.apache.org/hadoop/Distributions%20and%20Commercial%20Suppor
 [TIP]
 ====
 Hadoop 2.x is faster and includes features, such as short-circuit reads, which will help improve your HBase random read profile.
-Hadoop 2.x also includes important bug fixes that will improve your overall HBase experience.
-HBase 0.98 drops support for Hadoop 1.0, deprecates use of Hadoop 1.1+, and HBase 1.0 will not support Hadoop 1.x.
+Hadoop 2.x also includes important bug fixes that will improve your overall HBase experience. HBase does not support running with
+earlier versions of Hadoop. See the table below for requirements specific to different HBase versions.
 
 Hadoop 3.x is still in early access releases and has not yet been sufficiently tested by the HBase community for production use cases.
 ====
@@ -227,24 +207,21 @@ Use the following legend to interpret this table:
 * "X" = not supported
 * "NT" = Not tested
 
-[cols="1,1,1,1,1,1,1,1", options="header"]
+[cols="1,1,1,1,1", options="header"]
 |===
-| | HBase-0.94.x | HBase-0.98.x (Support for Hadoop 1.1+ is deprecated.) | HBase-1.0.x (Hadoop 1.x is NOT supported) | HBase-1.1.x | HBase-1.2.x | HBase-1.3.x | HBase-2.0.x
-|Hadoop-1.0.x  | X | X | X | X | X | X | X
-|Hadoop-1.1.x | S | NT | X | X | X | X | X
-|Hadoop-0.23.x | S | X | X | X | X | X | X
-|Hadoop-2.0.x-alpha | NT | X | X | X | X | X | X
-|Hadoop-2.1.0-beta | NT | X | X | X | X | X | X
-|Hadoop-2.2.0 | NT | S | NT | NT | X  | X | X
-|Hadoop-2.3.x | NT | S | NT | NT | X  | X | X
-|Hadoop-2.4.x | NT | S | S | S | S | S | X
-|Hadoop-2.5.x | NT | S | S | S | S | S | X
-|Hadoop-2.6.0 | X | X | X | X | X | X | X
-|Hadoop-2.6.1+ | NT | NT | NT | NT | S | S | S
-|Hadoop-2.7.0 | X | X | X | X | X | X | X
-|Hadoop-2.7.1+ | NT | NT | NT | NT | S | S | S
-|Hadoop-2.8.0 | X | X | X | X | X | X | X
-|Hadoop-3.0.0-alphax | NT | NT | NT | NT | NT | NT | NT
+| | HBase-1.1.x | HBase-1.2.x | HBase-1.3.x | HBase-2.0.x
+|Hadoop-2.0.x-alpha | X | X | X | X
+|Hadoop-2.1.0-beta | X | X | X | X
+|Hadoop-2.2.0 | NT | X  | X | X
+|Hadoop-2.3.x | NT | X  | X | X
+|Hadoop-2.4.x | S | S | S | X
+|Hadoop-2.5.x | S | S | S | X
+|Hadoop-2.6.0 | X | X | X | X
+|Hadoop-2.6.1+ | NT | S | S | S
+|Hadoop-2.7.0 | X | X | X | X
+|Hadoop-2.7.1+ | NT | S | S | S
+|Hadoop-2.8.0 | X | X | X | X
+|Hadoop-3.0.0-alphax | NT | NT | NT | NT
 |===
 
 .Hadoop Pre-2.6.1 and JDK 1.8 Kerberos
@@ -288,88 +265,6 @@ Make sure you replace the jar in HBase everywhere on your cluster.
 Hadoop version mismatch issues have various manifestations but often all looks like its hung up.
 ====
 
-[[hadoop2.hbase_0.94]]
-==== Apache HBase 0.94 with Hadoop 2
-
-To get 0.94.x to run on Hadoop 2.2.0, you need to change the hadoop 2 and protobuf versions in the _pom.xml_: Here is a diff with pom.xml changes:
-
-[source]
-----
-$ svn diff pom.xml
-Index: pom.xml
-===================================================================
---- pom.xml     (revision 1545157)
-+++ pom.xml     (working copy)
-@@ -1034,7 +1034,7 @@
-     <slf4j.version>1.4.3</slf4j.version>
-     <log4j.version>1.2.16</log4j.version>
-     <mockito-all.version>1.8.5</mockito-all.version>
--    <protobuf.version>2.4.0a</protobuf.version>
-+    <protobuf.version>2.5.0</protobuf.version>
-     <stax-api.version>1.0.1</stax-api.version>
-     <thrift.version>0.8.0</thrift.version>
-     <zookeeper.version>3.4.5</zookeeper.version>
-@@ -2241,7 +2241,7 @@
-         </property>
-       </activation>
-       <properties>
--        <hadoop.version>2.0.0-alpha</hadoop.version>
-+        <hadoop.version>2.2.0</hadoop.version>
-         <slf4j.version>1.6.1</slf4j.version>
-       </properties>
-       <dependencies>
-----
-
-The next step is to regenerate Protobuf files and assuming that the Protobuf has been installed:
-
-* Go to the HBase root folder, using the command line;
-* Type the following commands:
-+
-
-[source,bourne]
-----
-$ protoc -Isrc/main/protobuf --java_out=src/main/java src/main/protobuf/hbase.proto
-----
-+
-
-[source,bourne]
-----
-$ protoc -Isrc/main/protobuf --java_out=src/main/java src/main/protobuf/ErrorHandling.proto
-----
-
-
-Building against the hadoop 2 profile by running something like the following command:
-
-----
-$  mvn clean install assembly:single -Dhadoop.profile=2.0 -DskipTests
-----
-
-[[hadoop.hbase_0.94]]
-==== Apache HBase 0.92 and 0.94
-
-HBase 0.92 and 0.94 versions can work with Hadoop versions, 0.20.205, 0.22.x, 1.0.x, and 1.1.x.
-HBase-0.94 can additionally work with Hadoop-0.23.x and 2.x, but you may have to recompile the code using the specific maven profile (see top level pom.xml)
-
-[[hadoop.hbase_0.96]]
-==== Apache HBase 0.96
-
-As of Apache HBase 0.96.x, Apache Hadoop 1.0.x at least is required.
-Hadoop 2 is strongly encouraged (faster but also has fixes that help MTTR). We will no longer run properly on older Hadoops such as 0.20.205 or branch-0.20-append.
-Do not move to Apache HBase 0.96.x if you cannot upgrade your Hadoop. See link:http://search-hadoop.com/m/7vFVx4EsUb2[HBase, mail # dev - DISCUSS:
-                Have hbase require at least hadoop 1.0.0 in hbase 0.96.0?]
-
-[[hadoop.older.versions]]
-==== Hadoop versions 0.20.x - 1.x
-
-DO NOT use Hadoop versions older than 2.2.0 for HBase versions greater than 1.0. Check release documentation if you are using an older version of HBase for Hadoop related information. 
-
-[[hadoop.security]]
-==== Apache HBase on Secure Hadoop
-
-Apache HBase will run on any Hadoop 0.20.x that incorporates Hadoop security features as long as you do as suggested above and replace the Hadoop jar that ships with HBase with the secure version.
-If you want to read more about how to setup Secure HBase, see <<hbase.secure.configuration,hbase.secure.configuration>>.
-
-
 [[dfs.datanode.max.transfer.threads]]
 ==== `dfs.datanode.max.transfer.threads` (((dfs.datanode.max.transfer.threads)))
 
@@ -402,8 +297,8 @@ See also <<casestudies.max.transfer.threads,casestudies.max.transfer.threads>> a
 [[zookeeper.requirements]]
 === ZooKeeper Requirements
 
-ZooKeeper 3.4.x is required as of HBase 1.0.0.
-HBase makes use of the `multi` functionality that is only available since Zookeeper 3.4.0. The `hbase.zookeeper.useMulti` configuration property defaults to `true` in HBase 1.0.0.
+ZooKeeper 3.4.x is required.
+HBase makes use of the `multi` functionality that is only available since Zookeeper 3.4.0. The `hbase.zookeeper.useMulti` configuration property defaults to `true`.
 Refer to link:https://issues.apache.org/jira/browse/HBASE-12241[HBASE-12241 (The crash of regionServer when taking deadserver's replication queue breaks replication)] and link:https://issues.apache.org/jira/browse/HBASE-6775[HBASE-6775 (Use ZK.multi when available for HBASE-6710 0.92/0.94 compatibility fix)] for background.
 The property is deprecated and useMulti is always enabled in HBase 2.0.
 
@@ -590,8 +485,6 @@ Check them out especially if HBase had trouble starting.
 HBase also puts up a UI listing vital attributes.
 By default it's deployed on the Master host at port 16010 (HBase RegionServers listen on port 16020 by default and put up an informational HTTP server at port 16030). If the Master is running on a host named `master.example.org` on the default port, point your browser at pass:[http://master.example.org:16010] to see the web interface.
 
-Prior to HBase 0.98 the master UI was deployed on port 60010, and the HBase RegionServers UI on port 60030.
-
 Once HBase has started, see the <<shell_exercises,shell exercises>> section for how to create tables, add data, scan your insertions, and finally disable and drop your tables.
 
 To stop HBase after exiting the HBase shell enter
@@ -774,7 +667,7 @@ example9
 [[hbase_env]]
 ==== _hbase-env.sh_
 
-The following lines in the _hbase-env.sh_ file show how to set the `JAVA_HOME` environment variable (required for HBase 0.98.5 and newer) and set the heap to 4 GB (rather than the default value of 1 GB). If you copy and paste this example, be sure to adjust the `JAVA_HOME` to suit your environment.
+The following lines in the _hbase-env.sh_ file show how to set the `JAVA_HOME` environment variable (required for HBase) and set the heap to 4 GB (rather than the default value of 1 GB). If you copy and paste this example, be sure to adjust the `JAVA_HOME` to suit your environment.
 
 ----
 # The java implementation to use.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8e6f337/src/main/asciidoc/_chapters/cp.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc
index d0dcfef..2f5267f 100644
--- a/src/main/asciidoc/_chapters/cp.adoc
+++ b/src/main/asciidoc/_chapters/cp.adoc
@@ -675,15 +675,10 @@ public class SumEndPoint extends Sum.SumService implements Coprocessor, Coproces
 [source, java]
 ----
 Configuration conf = HBaseConfiguration.create();
-// Use below code for HBase version 1.x.x or above.
 Connection connection = ConnectionFactory.createConnection(conf);
 TableName tableName = TableName.valueOf("users");
 Table table = connection.getTable(tableName);
 
-//Use below code HBase version 0.98.xx or below.
-//HConnection connection = HConnectionManager.createConnection(conf);
-//HTableInterface table = connection.getTable("users");
-
 final Sum.SumRequest request = Sum.SumRequest.newBuilder().setFamily("salaryDet").setColumn("gross").build();
 try {
     Map<byte[], Long> results = table.coprocessorService(
@@ -760,15 +755,10 @@ Then you can read the configuration using code like the following:
 [source,java]
 ----
 Configuration conf = HBaseConfiguration.create();
-// Use below code for HBase version 1.x.x or above.
 Connection connection = ConnectionFactory.createConnection(conf);
 TableName tableName = TableName.valueOf("users");
 Table table = connection.getTable(tableName);
 
-//Use below code HBase version 0.98.xx or below.
-//HConnection connection = HConnectionManager.createConnection(conf);
-//HTableInterface table = connection.getTable("users");
-
 Get get = new Get(Bytes.toBytes("admin"));
 Result result = table.get(get);
 for (Cell c : result.rawCells()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8e6f337/src/main/asciidoc/_chapters/developer.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
index 5f72a6d..50b9c74 100644
--- a/src/main/asciidoc/_chapters/developer.adoc
+++ b/src/main/asciidoc/_chapters/developer.adoc
@@ -306,38 +306,27 @@ See the <<hbase.unittests.cmds,hbase.unittests.cmds>> section in <<hbase.unittes
 [[maven.build.hadoop]]
 ==== Building against various hadoop versions.
 
-As of 0.96, Apache HBase supports building against Apache Hadoop versions: 1.0.3, 2.0.0-alpha and 3.0.0-SNAPSHOT.
-By default, in 0.96 and earlier, we will build with Hadoop-1.0.x.
-As of 0.98, Hadoop 1.x is deprecated and Hadoop 2.x is the default.
-To change the version to build against, add a hadoop.profile property when you invoke +mvn+:
+HBase supports building against Apache Hadoop versions: 2.y and 3.y (early release artifacts). By default we build against Hadoop 2.x.
+
+To build against a specific release from the Hadoop 2.y line, set e.g. `-Dhadoop-two.version=2.6.3`.
 
 [source,bourne]
 ----
-mvn -Dhadoop.profile=1.0 ...
+mvn -Dhadoop-two.version=2.6.3 ...
 ----
 
-The above will build against whatever explicit hadoop 1.x version we have in our _pom.xml_ as our '1.0' version.
-Tests may not all pass so you may need to pass `-DskipTests` unless you are inclined to fix the failing tests.
-
-.'dependencyManagement.dependencies.dependency.artifactId' fororg.apache.hbase:${compat.module}:test-jar with value '${compat.module}'does not match a valid id pattern
-[NOTE]
-====
-You will see ERRORs like the above title if you pass the _default_ profile; e.g.
-if you pass +hadoop.profile=1.1+ when building 0.96 or +hadoop.profile=2.0+ when building hadoop 0.98; just drop the hadoop.profile stipulation in this case to get your build to run again.
-This seems to be a maven peculiarity that is probably fixable but we've not spent the time trying to figure it.
-====
-
-Similarly, for 3.0, you would just replace the profile value.
-Note that Hadoop-3.0.0-SNAPSHOT does not currently have a deployed maven artifact - you will need to build and install your own in your local maven repository if you want to run against this profile.
-
-In earlier versions of Apache HBase, you can build against older versions of Apache Hadoop, notably, Hadoop 0.22.x and 0.23.x.
-If you are running, for example HBase-0.94 and wanted to build against Hadoop 0.23.x, you would run with:
+To change the major release line of Hadoop we build against, add a hadoop.profile property when you invoke +mvn+:
 
 [source,bourne]
 ----
-mvn -Dhadoop.profile=22 ...
+mvn -Dhadoop.profile=3.0 ...
 ----
 
+The above will build against whatever explicit hadoop 3.y version we have in our _pom.xml_ as our '3.0' version.
+Tests may not all pass so you may need to pass `-DskipTests` unless you are inclined to fix the failing tests.
+
+To pick a particular Hadoop 3.y release, you'd set e.g. `-Dhadoop-three.version=3.0.0-alpha1`.
+
 [[build.protobuf]]
 ==== Build Protobuf
 
@@ -426,27 +415,6 @@ HBase 1.x requires Java 7 to build.
 See <<java,java>> for Java requirements per HBase release.
 ====
 
-=== Building against HBase 0.96-0.98
-
-HBase 0.96.x will run on Hadoop 1.x or Hadoop 2.x.
-HBase 0.98 still runs on both, but HBase 0.98 deprecates use of Hadoop 1.
-HBase 1.x will _not_                run on Hadoop 1.
-In the following procedures, we make a distinction between HBase 1.x builds and the awkward process involved building HBase 0.96/0.98 for either Hadoop 1 or Hadoop 2 targets.
-
-You must choose which Hadoop to build against.
-It is not possible to build a single HBase binary that runs against both Hadoop 1 and Hadoop 2.
-Hadoop is included in the build, because it is needed to run HBase in standalone mode.
-Therefore, the set of modules included in the tarball changes, depending on the build target.
-To determine which HBase you have, look at the HBase version.
-The Hadoop version is embedded within it.
-
-Maven, our build system, natively does not allow a single product to be built against different dependencies.
-Also, Maven cannot change the set of included modules and write out the correct _pom.xml_ files with appropriate dependencies, even using two build targets, one for Hadoop 1 and another for Hadoop 2.
-A prerequisite step is required, which takes as input the current _pom.xml_s and generates Hadoop 1 or Hadoop 2 versions using a script in the _dev-tools/_ directory, called _generate-hadoopX-poms.sh_                where [replaceable]_X_ is either `1` or `2`.
-You then reference these generated poms when you build.
-For now, just be aware of the difference between HBase 1.x builds and those of HBase 0.96-0.98.
-This difference is important to the build instructions.
-
 [[maven.settings.xml]]
 .Example _~/.m2/settings.xml_ File
 ====
@@ -496,9 +464,7 @@ For the build to sign them for you, you a properly configured _settings.xml_ in
 [[maven.release]]
 === Making a Release Candidate
 
-NOTE: These instructions are for building HBase 1.0.x.
-For building earlier versions, e.g. 0.98.x, the process is different.
-See this section under the respective release documentation folders.
+NOTE: These instructions are for building HBase 1.y.z
 
 .Point Releases
 If you are making a point release (for example to quickly address a critical incompatibility or security problem) off of a release branch instead of a development branch, the tagging instructions are slightly different.
@@ -1440,8 +1406,6 @@ The following interface classifications are commonly used:
 
 `@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)`::
   APIs for HBase coprocessor writers.
-  As of HBase 0.92/0.94/0.96/0.98 this api is still unstable.
-  No guarantees on compatibility with future versions.
 
 No `@InterfaceAudience` Classification::
   Packages without an `@InterfaceAudience` label are considered private.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8e6f337/src/main/asciidoc/_chapters/upgrading.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index 46f637d..7210040 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -145,6 +145,9 @@ HBase Private API::
 [[hbase.versioning.pre10]]
 === Pre 1.0 versions
 
+.HBase Pre-1.0 versions are all EOM
+NOTE: For new installations, do not deploy 0.94.y, 0.96.y, or 0.98.y.  Deploy our stable version. See link:https://issues.apache.org/jira/browse/HBASE-11642[EOL 0.96], link:https://issues.apache.org/jira/browse/HBASE-16215[clean up of EOM releases], and link:http://www.apache.org/dist/hbase/[the header of our downloads].
+
 Before the semantic versioning scheme pre-1.0, HBase tracked either Hadoop's versions (0.2x) or 0.9x versions. If you are into the arcane, checkout our old wiki page on link:http://wiki.apache.org/hadoop/Hbase/HBaseVersions[HBase Versioning] which tries to connect the HBase version dots. Below sections cover ONLY the releases before 1.0.
 
 [[hbase.development.series]]
@@ -260,9 +263,6 @@ A rolling upgrade from 0.94.x directly to 0.98.x does not work. The upgrade path
 
 ==== The "Singularity"
 
-.HBase 0.96.x was EOL'd, September 1st, 2014
-NOTE: Do not deploy 0.96.x  Deploy at least 0.98.x. See link:https://issues.apache.org/jira/browse/HBASE-11642[EOL 0.96].
-
 You will have to stop your old 0.94.x cluster completely to upgrade. If you are replicating between clusters, both clusters will have to go down to upgrade. Make sure it is a clean shutdown. The less WAL files around, the faster the upgrade will run (the upgrade will split any log files it finds in the filesystem as part of the upgrade process). All clients must be upgraded to 0.96 too.
 
 The API has changed. You will need to recompile your code against 0.96 and you may need to adjust applications to go against new APIs (TODO: List of changes).

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8e6f337/src/main/site/site.xml
----------------------------------------------------------------------
diff --git a/src/main/site/site.xml b/src/main/site/site.xml
index 21ab095..419cb8a 100644
--- a/src/main/site/site.xml
+++ b/src/main/site/site.xml
@@ -121,11 +121,6 @@
         <item name="X-Ref" href="1.1/xref/index.html" target="_blank" />
         <item name="Ref Guide (single-page)" href="1.1/book.html" target="_blank" />
       </item>
-      <item name="0.94 Documentation">
-        <item name="API" href="0.94/apidocs/index.html" target="_blank" />
-        <item name="X-Ref" href="0.94/xref/index.html" target="_blank" />
-        <item name="Ref Guide (single-page)" href="0.94/book.html" target="_blank" />
-      </item>
     </menu>
     <menu name="ASF">
       <item name="Apache Software Foundation" href="http://www.apache.org/foundation/" target="_blank" />


[26/40] hbase git commit: HBASE-17943 Addendum increases the threshold value of in-memory compaction for TestWalAndCompactingMemStoreFlush

Posted by sy...@apache.org.
HBASE-17943 Addendum increases the threshold value of in-memory compaction for TestWalAndCompactingMemStoreFlush


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

Branch: refs/heads/hbase-12439
Commit: 9053ec6fe6505eba4f14adfdd83329511e4a77f0
Parents: e95cf47
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Sat Apr 22 20:47:55 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Sat Apr 22 20:47:55 2017 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/TestWalAndCompactingMemStoreFlush.java   | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9053ec6f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
index 2c16399..3b2ebe2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
@@ -36,7 +36,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
@@ -136,7 +135,7 @@ public class TestWalAndCompactingMemStoreFlush {
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
         FlushNonSloppyStoresFirstPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 * 1024);
-    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.25);
+    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
     // set memstore to do data compaction
     conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
         String.valueOf(MemoryCompactionPolicy.EAGER));
@@ -771,7 +770,7 @@ public class TestWalAndCompactingMemStoreFlush {
         FlushNonSloppyStoresFirstPolicy.class.getName());
     conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
         75 * 1024);
-    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
+    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.8);
     // set memstore to do index compaction with merge
     conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
         String.valueOf(MemoryCompactionPolicy.BASIC));


[21/40] hbase git commit: HBASE-17941 CellArrayMap#getCell may throw IndexOutOfBoundsException

Posted by sy...@apache.org.
HBASE-17941 CellArrayMap#getCell may throw IndexOutOfBoundsException

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/33dadc1a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/33dadc1a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/33dadc1a

Branch: refs/heads/hbase-12439
Commit: 33dadc1a941a536742799a46444c67a1ed66d124
Parents: ea3a27b
Author: s9514171 <s9...@gmail.com>
Authored: Thu Apr 20 14:54:52 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Apr 21 11:35:39 2017 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/33dadc1a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayMap.java
index 605fea2..898e469 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayMap.java
@@ -48,7 +48,7 @@ public class CellArrayMap extends CellFlatMap {
 
   @Override
   protected Cell getCell(int i) {
-    if( (i < minCellIdx) && (i >= maxCellIdx) ) return null;
+    if( (i < minCellIdx) || (i >= maxCellIdx) ) return null;
     return block[i];
   }
 }


[22/40] hbase git commit: HBASE-17864: Implement async snapshot/cloneSnapshot/restoreSnapshot methods

Posted by sy...@apache.org.
HBASE-17864: Implement async snapshot/cloneSnapshot/restoreSnapshot 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/d39f40e7
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d39f40e7
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d39f40e7

Branch: refs/heads/hbase-12439
Commit: d39f40e787ecab54ee597ac4463bbbd2f5e944d9
Parents: 33dadc1
Author: huzheng <op...@gmail.com>
Authored: Thu Apr 20 18:59:43 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Apr 21 18:57:43 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  88 +++++++++++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    | 145 +++++++++++++++++++
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   3 +-
 .../org/apache/hadoop/hbase/HConstants.java     |   4 +
 .../hbase/client/TestAsyncSnapshotAdminApi.java | 112 ++++++++++++++
 5 files changed, 351 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d39f40e7/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 5d2955f..b7c60dd 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
@@ -573,4 +573,92 @@ public interface AsyncAdmin {
    *         {@link CompletableFuture}.
    */
   CompletableFuture<List<TableCFs>> listReplicatedTableCFs();
+
+  /**
+   * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
+   * taken. If the table is disabled, an offline snapshot is taken. Snapshots are considered unique
+   * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
+   * a different type or with different parameters) will fail with a
+   * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
+   * naming. Snapshot names follow the same naming constraints as tables in HBase. See
+   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
+   * @param snapshotName name of the snapshot to be created
+   * @param tableName name of the table for which snapshot is created
+   */
+  CompletableFuture<Void> snapshot(String snapshotName, TableName tableName);
+
+  /**
+   * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
+   * snapshot</b>. Attempts to take a snapshot with the same name (even a different type or with
+   * different parameters) will fail with a
+   * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
+   * naming. Snapshot names follow the same naming constraints as tables in HBase. See
+   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
+   * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
+   *          snapshots stored on the cluster
+   * @param tableName name of the table to snapshot
+   * @param type type of snapshot to take
+   */
+  CompletableFuture<Void> snapshot(final String snapshotName, final TableName tableName,
+      SnapshotType type);
+
+  /**
+   * Take a snapshot and wait for the server to complete that snapshot asynchronously. Only a single
+   * snapshot should be taken at a time for an instance of HBase, or results may be undefined (you
+   * can tell multiple HBase clusters to snapshot at the same time, but only one at a time for a
+   * single cluster). Snapshots are considered unique based on <b>the name of the snapshot</b>.
+   * Attempts to take a snapshot with the same name (even a different type or with different
+   * parameters) will fail with a {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException}
+   * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
+   * HBase. See {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
+   * You should probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} unless you
+   * are sure about the type of snapshot that you want to take.
+   * @param snapshot snapshot to take
+   */
+  CompletableFuture<Void> snapshot(SnapshotDescription snapshot);
+
+  /**
+   * Check the current state of the passed snapshot. There are three possible states:
+   * <ol>
+   * <li>running - returns <tt>false</tt></li>
+   * <li>finished - returns <tt>true</tt></li>
+   * <li>finished with error - throws the exception that caused the snapshot to fail</li>
+   * </ol>
+   * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
+   * run/started since the snapshot your are checking, you will recieve an
+   * {@link org.apache.hadoop.hbase.snapshot.UnknownSnapshotException}.
+   * @param snapshot description of the snapshot to check
+   * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
+   *         running
+   */
+  CompletableFuture<Boolean> isSnapshotFinished(final SnapshotDescription snapshot);
+
+  /**
+   * Restore the specified snapshot on the original table. (The table must be disabled) If the
+   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a
+   * snapshot of the current table is taken before executing the restore operation. In case of
+   * restore failure, the failsafe snapshot will be restored. If the restore completes without
+   * problem the failsafe snapshot is deleted.
+   * @param snapshotName name of the snapshot to restore
+   */
+  CompletableFuture<Void> restoreSnapshot(String snapshotName);
+
+  /**
+   * Restore the specified snapshot on the original table. (The table must be disabled) If
+   * 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before
+   * executing the restore operation. In case of restore failure, the failsafe snapshot will be
+   * restored. If the restore completes without problem the failsafe snapshot is deleted. The
+   * failsafe snapshot name is configurable by using the property
+   * "hbase.snapshot.restore.failsafe.name".
+   * @param snapshotName name of the snapshot to restore
+   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
+   */
+  CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot);
+
+  /**
+   * Create a new table by cloning the snapshot content.
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   */
+  CompletableFuture<Void> cloneSnapshot(final String snapshotName, final TableName tableName);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d39f40e7/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 eae4089..54e1e8b 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
@@ -37,6 +37,8 @@ import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import io.netty.util.Timeout;
+import io.netty.util.TimerTask;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -50,6 +52,7 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
 import org.apache.hadoop.hbase.TableNotFoundException;
@@ -77,6 +80,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegion
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
@@ -110,6 +114,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
@@ -123,10 +129,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegion
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
 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.SnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
 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;
@@ -145,7 +155,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
 
@@ -1387,6 +1400,138 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
     return future;
   }
 
+  @Override
+  public CompletableFuture<Void> snapshot(String snapshotName, TableName tableName) {
+    return snapshot(snapshotName, tableName, SnapshotType.FLUSH);
+  }
+
+  @Override
+  public CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
+      SnapshotType type) {
+    return snapshot(new SnapshotDescription(snapshotName, tableName, type));
+  }
+
+  @Override
+  public CompletableFuture<Void> snapshot(SnapshotDescription snapshotDesc) {
+    HBaseProtos.SnapshotDescription snapshot =
+        ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc);
+    try {
+      ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
+    } catch (IllegalArgumentException e) {
+      return failedFuture(e);
+    }
+    CompletableFuture<Void> future = new CompletableFuture<>();
+    final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot).build();
+    this.<Long> newMasterCaller()
+        .action((controller, stub) -> this.<SnapshotRequest, SnapshotResponse, Long> call(
+          controller, stub, request, (s, c, req, done) -> s.snapshot(c, req, done),
+          resp -> resp.getExpectedTimeout()))
+        .call().whenComplete((expectedTimeout, err) -> {
+          if (err != null) {
+            future.completeExceptionally(err);
+            return;
+          }
+          TimerTask pollingTask = new TimerTask() {
+            int tries = 0;
+            long startTime = EnvironmentEdgeManager.currentTime();
+            long endTime = startTime + expectedTimeout;
+            long maxPauseTime = expectedTimeout / maxAttempts;
+
+            @Override
+            public void run(Timeout timeout) throws Exception {
+              if (EnvironmentEdgeManager.currentTime() < endTime) {
+                isSnapshotFinished(snapshotDesc).whenComplete((done, err) -> {
+                  if (err != null) {
+                    future.completeExceptionally(err);
+                  } else if (done) {
+                    future.complete(null);
+                  } else {
+                    // retry again after pauseTime.
+                    long pauseTime = ConnectionUtils
+                        .getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries);
+                    pauseTime = Math.min(pauseTime, maxPauseTime);
+                    AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime,
+                      TimeUnit.MILLISECONDS);
+                  }
+                });
+              } else {
+                future.completeExceptionally(new SnapshotCreationException(
+                    "Snapshot '" + snapshot.getName() + "' wasn't completed in expectedTime:"
+                        + expectedTimeout + " ms",
+                    snapshotDesc));
+              }
+            }
+          };
+          AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS);
+        });
+    return future;
+  }
+
+  @Override
+  public CompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot) {
+    return this.<Boolean> newMasterCaller()
+        .action((controller, stub) -> this
+            .<IsSnapshotDoneRequest, IsSnapshotDoneResponse, Boolean> call(controller, stub,
+              IsSnapshotDoneRequest.newBuilder()
+                  .setSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot)).build(),
+              (s, c, req, done) -> s.isSnapshotDone(c, req, done), resp -> resp.getDone()))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<Void> restoreSnapshot(String snapshotName) {
+    boolean takeFailSafeSnapshot = this.connection.getConfiguration().getBoolean(
+      HConstants.SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT,
+      HConstants.DEFAULT_SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT);
+    return restoreSnapshot(snapshotName, takeFailSafeSnapshot);
+  }
+
+  @Override
+  public CompletableFuture<Void> restoreSnapshot(String snapshotName,
+      boolean takeFailSafeSnapshot) {
+    // TODO It depend on listSnapshots() method.
+    return failedFuture(new UnsupportedOperationException("restoreSnapshot do not supported yet"));
+  }
+
+  @Override
+  public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName) {
+    CompletableFuture<Void> future = new CompletableFuture<>();
+    tableExists(tableName).whenComplete((exists, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+      } else if (exists) {
+        future.completeExceptionally(new TableExistsException(tableName));
+      } else {
+        internalRestoreSnapshot(snapshotName, tableName).whenComplete((ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
+      }
+    });
+    return future;
+  }
+
+  private CompletableFuture<Void> internalRestoreSnapshot(String snapshotName,
+      TableName tableName) {
+    HBaseProtos.SnapshotDescription snapshot = HBaseProtos.SnapshotDescription.newBuilder()
+        .setName(snapshotName).setTable(tableName.getNameAsString()).build();
+    try {
+      ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
+    } catch (IllegalArgumentException e) {
+      return failedFuture(e);
+    }
+    return this.<Void> newMasterCaller()
+        .action((controller, stub) -> this
+            .<RestoreSnapshotRequest, RestoreSnapshotResponse, Void> call(controller, stub,
+              RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
+                  .setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce()).build(),
+              (s, c, req, done) -> s.restoreSnapshot(c, req, done), resp -> null))
+        .call();
+  }
+
   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/d39f40e7/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 8063070..e55a95d 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
@@ -2498,7 +2498,8 @@ public class HBaseAdmin implements Admin {
   public void restoreSnapshot(final String snapshotName)
       throws IOException, RestoreSnapshotException {
     boolean takeFailSafeSnapshot =
-      conf.getBoolean("hbase.snapshot.restore.take.failsafe.snapshot", false);
+        conf.getBoolean(HConstants.SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT,
+          HConstants.DEFAULT_SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT);
     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d39f40e7/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 eff5690..c0dbfe4 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
@@ -1351,6 +1351,10 @@ public final class HConstants {
   public static final String DEFAULT_TEMPORARY_HDFS_DIRECTORY = "/user/"
       + System.getProperty("user.name") + "/hbase-staging";
 
+  public static final String SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT =
+      "hbase.snapshot.restore.take.failsafe.snapshot";
+  public static final boolean DEFAULT_SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT = false;
+
   private HConstants() {
     // Can't be instantiated with this ctor.
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d39f40e7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java
new file mode 100644
index 0000000..0eb3881
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSnapshotAdminApi.java
@@ -0,0 +1,112 @@
+/**
+ * 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.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Collections;
+import java.util.List;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestAsyncSnapshotAdminApi extends TestAsyncAdminBase {
+
+  @Test
+  public void testTakeSnapshot() throws Exception {
+    String snapshotName1 = "snapshotName1";
+    String snapshotName2 = "snapshotName2";
+    TableName tableName = TableName.valueOf("testTakeSnapshot");
+    Admin syncAdmin = TEST_UTIL.getAdmin();
+
+    try {
+      Table table = TEST_UTIL.createTable(tableName, Bytes.toBytes("f1"));
+      for (int i = 0; i < 3000; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(Bytes.toBytes("f1"), Bytes.toBytes("cq"),
+          Bytes.toBytes(i)));
+      }
+
+      admin.snapshot(snapshotName1, tableName).get();
+      admin.snapshot(snapshotName2, tableName).get();
+      List<SnapshotDescription> snapshots = syncAdmin.listSnapshots();
+      Collections.sort(snapshots, (snap1, snap2) -> {
+        Assert.assertNotNull(snap1);
+        Assert.assertNotNull(snap1.getName());
+        Assert.assertNotNull(snap2);
+        Assert.assertNotNull(snap2.getName());
+        return snap1.getName().compareTo(snap2.getName());
+      });
+
+      Assert.assertEquals(snapshotName1, snapshots.get(0).getName());
+      Assert.assertEquals(tableName, snapshots.get(0).getTableName());
+      Assert.assertEquals(SnapshotType.FLUSH, snapshots.get(0).getType());
+      Assert.assertEquals(snapshotName2, snapshots.get(1).getName());
+      Assert.assertEquals(tableName, snapshots.get(1).getTableName());
+      Assert.assertEquals(SnapshotType.FLUSH, snapshots.get(1).getType());
+    } finally {
+      syncAdmin.deleteSnapshot(snapshotName1);
+      syncAdmin.deleteSnapshot(snapshotName2);
+      TEST_UTIL.deleteTable(tableName);
+    }
+  }
+
+  @Test
+  public void testCloneSnapshot() throws Exception {
+    String snapshotName1 = "snapshotName1";
+    TableName tableName = TableName.valueOf("testCloneSnapshot");
+    TableName tableName2 = TableName.valueOf("testCloneSnapshot2");
+    Admin syncAdmin = TEST_UTIL.getAdmin();
+
+    try {
+      Table table = TEST_UTIL.createTable(tableName, Bytes.toBytes("f1"));
+      for (int i = 0; i < 3000; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(Bytes.toBytes("f1"), Bytes.toBytes("cq"),
+          Bytes.toBytes(i)));
+      }
+
+      admin.snapshot(snapshotName1, tableName).get();
+      List<SnapshotDescription> snapshots = syncAdmin.listSnapshots();
+      Assert.assertEquals(snapshots.size(), 1);
+      Assert.assertEquals(snapshotName1, snapshots.get(0).getName());
+      Assert.assertEquals(tableName, snapshots.get(0).getTableName());
+      Assert.assertEquals(SnapshotType.FLUSH, snapshots.get(0).getType());
+
+      // cloneSnapshot into a existed table.
+      boolean failed = false;
+      try {
+        admin.cloneSnapshot(snapshotName1, tableName).get();
+      } catch (Exception e) {
+        failed = true;
+      }
+      Assert.assertTrue(failed);
+
+      // cloneSnapshot into a new table.
+      Assert.assertTrue(!syncAdmin.tableExists(tableName2));
+      admin.cloneSnapshot(snapshotName1, tableName2).get();
+      syncAdmin.tableExists(tableName2);
+    } finally {
+      syncAdmin.deleteSnapshot(snapshotName1);
+      TEST_UTIL.deleteTable(tableName);
+    }
+  }
+}
\ No newline at end of file


[03/40] hbase git commit: Add hbasecon asia and next weeks visa meetup

Posted by sy...@apache.org.
Add hbasecon asia and next weeks visa meetup


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

Branch: refs/heads/hbase-12439
Commit: b35121d904e7e16a04e60a6471d05fb15d598acf
Parents: 5eda5fb
Author: Michael Stack <st...@apache.org>
Authored: Mon Apr 17 22:19:49 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Apr 17 22:20:04 2017 -0700

----------------------------------------------------------------------
 src/main/site/xdoc/index.xml | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b35121d9/src/main/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/index.xml b/src/main/site/xdoc/index.xml
index 83c9f01..1848d40 100644
--- a/src/main/site/xdoc/index.xml
+++ b/src/main/site/xdoc/index.xml
@@ -83,7 +83,9 @@ Apache HBase is an open-source, distributed, versioned, non-relational database
 </section>
 
      <section name="News">
+       <p>August 4th, 2017 <a href="https://easychair.org/cfp/HBaseConAsia2017">HBaseCon Asia 2017</a> @ the Huawei Campus in Shenzhen, China</p>
        <p>June 12th, 2017 <a href="https://easychair.org/cfp/hbasecon2017">HBaseCon2017</a> at the Crittenden Buildings on the Google Mountain View Campus</p>
+       <p>April 25th, 2017 <a href="https://www.meetup.com/hbaseusergroup/events/239291716/">Meetup</a> @ Visa in Palo Alto</p>
         <p>December 8th, 2016 <a href="https://www.meetup.com/hbaseusergroup/events/235542241/">Meetup@Splice</a> in San Francisco</p>
        <p>September 26th, 2016 <a href="http://www.meetup.com/HBase-NYC/events/233024937/">HBaseConEast2016</a> at Google in Chelsea, NYC</p>
          <p>May 24th, 2016 <a href="http://www.hbasecon.com/">HBaseCon2016</a> at The Village, 969 Market, San Francisco</p>


[40/40] hbase git commit: HBASE-17873 Change the IA.Public annotation to IA.Private for unstable API

Posted by sy...@apache.org.
HBASE-17873 Change the IA.Public annotation to IA.Private for unstable API

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/177344cd
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/177344cd
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/177344cd

Branch: refs/heads/hbase-12439
Commit: 177344cdbf8a08a6d225312b2327ff1ea8a21067
Parents: 053e615
Author: zhangduo <zh...@apache.org>
Authored: Thu Apr 6 11:56:44 2017 +0800
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Apr 26 16:39:11 2017 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java  | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/177344cd/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 3343c7a..352ef1b 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
@@ -37,9 +37,12 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.util.Pair;
 
 /**
- *  The asynchronous administrative API for HBase.
+ * The asynchronous administrative API for HBase.
+ * <p>
+ * This feature is still under development, so marked as IA.Private. Will change to public when
+ * done. Use it with caution.
  */
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 public interface AsyncAdmin {
 
   /**


[13/40] 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/972e8c8c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/972e8c8c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/972e8c8c

Branch: refs/heads/hbase-12439
Commit: 972e8c8c296d38507077b98c8fc2a33eda9fce66
Parents: 6e962d6
Author: Ramkrishna <ra...@intel.com>
Authored: Wed Apr 19 15:28:03 2017 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Wed Apr 19 15:28:03 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     | 119 +++---
 .../TestMemstoreLABWithoutPool.java             | 168 ++++++++
 .../hbase/regionserver/TestRecoveredEdits.java  |   1 +
 .../hbase/regionserver/TestRegionIncrement.java |   1 +
 .../regionserver/TestReversibleScanners.java    |   7 +-
 .../hadoop/hbase/regionserver/TestStore.java    |   1 +
 .../TestStoreFileRefresherChore.java            |   1 +
 .../hbase/regionserver/TestWALLockup.java       |   1 +
 .../TestWALMonotonicallyIncreasingSeqId.java    |   1 +
 .../regionserver/wal/AbstractTestFSWAL.java     |   2 +
 .../hbase/regionserver/wal/TestDurability.java  |   3 +
 .../hbase/regionserver/wal/TestFSHLog.java      |   4 +-
 44 files changed, 1055 insertions(+), 519 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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..d550148
--- /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 static 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/972e8c8c/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 d14571b..c197418 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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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 acf2af0..c0ddbfc 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
@@ -95,6 +95,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;
@@ -2424,6 +2426,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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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/972e8c8c/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 58dbe8d..543ca6c 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
@@ -101,6 +101,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/972e8c8c/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/972e8c8c/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/972e8c8c/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);


[32/40] hbase git commit: HBASE-15143 Procedure v2 - Web UI displaying queues

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-protocol-shaded/src/main/protobuf/LockService.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
index 0df7f2e..1898e68 100644
--- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
@@ -25,6 +25,7 @@ option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
 import "HBase.proto";
+import "Procedure.proto";
 
 enum LockType {
   EXCLUSIVE = 1;
@@ -70,6 +71,27 @@ message LockProcedureData {
   optional bool is_master_lock = 6 [default = false];
 }
 
+enum ResourceType {
+  RESOURCE_TYPE_SERVER = 1;
+  RESOURCE_TYPE_NAMESPACE = 2;
+  RESOURCE_TYPE_TABLE = 3;
+  RESOURCE_TYPE_REGION = 4;
+}
+
+message WaitingProcedure {
+  required LockType lock_type = 1;
+  required Procedure procedure = 2;
+}
+
+message LockInfo {
+  required ResourceType resource_type = 1;
+  optional string resource_name = 2;
+  required LockType lock_type = 3;
+  optional Procedure exclusive_lock_owner_procedure = 4;
+  optional int32 shared_lock_count = 5;
+  repeated WaitingProcedure waitingProcedures = 6;
+}
+
 service LockService {
   /** Acquire lock on namespace/table/region */
   rpc RequestLock(LockRequest) returns(LockResponse);

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 d7d51e2..0c3da02 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -30,6 +30,7 @@ import "HBase.proto";
 import "Client.proto";
 import "ClusterStatus.proto";
 import "ErrorHandling.proto";
+import "LockService.proto";
 import "Procedure.proto";
 import "Quota.proto";
 import "Replication.proto";
@@ -534,6 +535,13 @@ message ListProceduresResponse {
   repeated Procedure procedure = 1;
 }
 
+message ListLocksRequest {
+}
+
+message ListLocksResponse {
+  repeated LockInfo lock = 1;
+}
+
 message SetQuotaRequest {
   optional string user_name = 1;
   optional string user_group = 2;
@@ -888,6 +896,9 @@ service MasterService {
   rpc ListProcedures(ListProceduresRequest)
     returns(ListProceduresResponse);
 
+  rpc ListLocks(ListLocksRequest)
+    returns(ListLocksResponse);
+
   /** Add a replication peer */
   rpc AddReplicationPeer(AddReplicationPeerRequest)
     returns(AddReplicationPeerResponse);

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index 36d5112..e1a47c5 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -125,7 +125,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
                 <ul class="nav navbar-nav">
                 <li class="active"><a href="/">Home</a></li>
                 <li><a href="/tablesDetailed.jsp">Table Details</a></li>
-                <li><a href="/procedures.jsp">Procedures</a></li>
+                <li><a href="/procedures.jsp">Procedures &amp; Locks</a></li>
                 <li><a href="/logs/">Local Logs</a></li>
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/dump">Debug Dump</a></li>

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index aab852c..ad8aa14 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
@@ -983,6 +984,24 @@ public interface MasterObserver extends Coprocessor {
       List<ProcedureInfo> procInfoList) throws IOException {}
 
   /**
+   * Called before a listLocks request has been processed.
+   * @param ctx the environment to interact with the framework and master
+   * @throws IOException if something went wrong
+   */
+  default void preListLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException {}
+
+  /**
+   * Called after a listLocks request has been processed.
+   * @param ctx the environment to interact with the framework and master
+   * @param lockInfoList the list of locks about to be returned
+   * @throws IOException if something went wrong
+   */
+  default void postListLocks(
+      ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<LockInfo> lockInfoList) throws IOException {}
+
+  /**
    * Called prior to moving a given region from one region server to another.
    * @param ctx the environment to interact with the framework and master
    * @param region the HRegionInfo

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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..e4ba285 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
@@ -65,7 +65,6 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.RegionStateListener;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
@@ -113,6 +112,7 @@ import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
 import org.apache.hadoop.hbase.master.procedure.MergeTableRegionsProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
@@ -128,6 +128,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
@@ -979,7 +980,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   void initQuotaManager() throws IOException {
     MasterQuotaManager quotaManager = new MasterQuotaManager(this);
-    this.assignmentManager.setRegionStateListener((RegionStateListener)quotaManager);
+    this.assignmentManager.setRegionStateListener(quotaManager);
     quotaManager.start();
     this.quotaManager = quotaManager;
   }
@@ -1141,8 +1142,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureStore = new WALProcedureStore(conf, walDir.getFileSystem(conf), walDir,
         new MasterProcedureEnv.WALStoreLeaseRecovery(this));
     procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
-    procedureExecutor = new ProcedureExecutor(conf, procEnv, procedureStore,
-        procEnv.getProcedureScheduler());
+    MasterProcedureScheduler procedureScheduler = procEnv.getProcedureScheduler();
+    procedureExecutor = new ProcedureExecutor<>(conf, procEnv, procedureStore, procedureScheduler);
     configurationManager.registerObserver(procEnv);
 
     final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
@@ -2909,6 +2910,34 @@ public class HMaster extends HRegionServer implements MasterServices {
     return procInfoList;
   }
 
+  private Map<Long, ProcedureInfo> getProcedureInfos() {
+    final List<ProcedureInfo> list = procedureExecutor.listProcedures();
+    final Map<Long, ProcedureInfo> map = new HashMap<>();
+
+    for (ProcedureInfo procedureInfo : list) {
+      map.put(procedureInfo.getProcId(), procedureInfo);
+    }
+
+    return map;
+  }
+
+  @Override
+  public List<LockInfo> listLocks() throws IOException {
+    if (cpHost != null) {
+      cpHost.preListLocks();
+    }
+
+    MasterProcedureScheduler procedureScheduler = procedureExecutor.getEnvironment().getProcedureScheduler();
+
+    final List<LockInfo> lockInfoList = procedureScheduler.listLocks();
+
+    if (cpHost != null) {
+      cpHost.postListLocks(lockInfoList);
+    }
+
+    return lockInfoList;
+  }
+
   /**
    * Returns the list of table descriptors that match the specified request
    * @param namespace the namespace to query, or null if querying for all

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 8a7a387..2f5e66e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.User;
@@ -706,6 +707,26 @@ public class MasterCoprocessorHost
     });
   }
 
+  public boolean preListLocks() throws IOException {
+    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        oserver.preListLocks(ctx);
+      }
+    });
+  }
+
+  public void postListLocks(final List<LockInfo> lockInfoList) throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        oserver.postListLocks(ctx, lockInfoList);
+      }
+    });
+  }
+
   public boolean preMove(final HRegionInfo region, final ServerName srcServer,
       final ServerName destServer) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 9af8f45..40c4a71 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
@@ -56,8 +56,8 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -86,129 +86,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockH
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
-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.SetCleanerChoreRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
-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.SetSplitOrMergeEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
-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;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -1108,7 +987,7 @@ public class MasterRpcServices extends RSRpcServices
         }
         master.getMasterProcedureExecutor().removeResult(request.getProcId());
       } else {
-        Procedure proc = v.getSecond();
+        Procedure<?> proc = v.getSecond();
         if (proc == null) {
           builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
         } else {
@@ -1160,7 +1039,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       final ListProceduresResponse.Builder response = ListProceduresResponse.newBuilder();
       for (ProcedureInfo p: master.listProcedures()) {
-        response.addProcedure(ProcedureUtil.convertToProtoProcedure(p));
+        response.addProcedure(ProtobufUtil.toProtoProcedure(p));
       }
       return response.build();
     } catch (IOException e) {
@@ -1169,6 +1048,23 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public ListLocksResponse listLocks(
+      RpcController controller,
+      ListLocksRequest request) throws ServiceException {
+    try {
+      final ListLocksResponse.Builder builder = ListLocksResponse.newBuilder();
+
+      for (LockInfo lockInfo: master.listLocks()) {
+        builder.addLock(ProtobufUtil.toProtoLockInfo(lockInfo));
+      }
+
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController c,
       ListTableDescriptorsByNamespaceRequest request) throws ServiceException {
     try {
@@ -1459,7 +1355,6 @@ public class MasterRpcServices extends RSRpcServices
         throw new UnknownRegionException(Bytes.toString(regionName));
       }
 
-      if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
       HRegionInfo hri = pair.getFirst();
       if (master.cpHost != null) {
         if (master.cpHost.preUnassign(hri, force)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 66758f8..4924d72 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -39,12 +39,12 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-
 import com.google.protobuf.Service;
 
 /**
@@ -353,6 +353,13 @@ public interface MasterServices extends Server {
   public List<ProcedureInfo> listProcedures() throws IOException;
 
   /**
+   * List locks
+   * @return lock list
+   * @throws IOException
+   */
+  public List<LockInfo> listLocks() throws IOException;
+
+  /**
    * Get list of table descriptors by namespace
    * @param name namespace name
    * @return descriptors

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
index 3cad51c..512f7e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
@@ -204,6 +204,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
    * WALs.
    * @return false, so procedure framework doesn't mark this procedure as failure.
    */
+  @Override
   protected boolean setTimeoutFailure(final MasterProcedureEnv env) {
     synchronized (event) {
       if (LOG.isDebugEnabled()) LOG.debug("Timeout failure " + this.event);
@@ -231,7 +232,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
   }
 
   @Override
-  protected Procedure[] execute(final MasterProcedureEnv env) throws ProcedureSuspendedException {
+  protected Procedure<?>[] execute(final MasterProcedureEnv env) throws ProcedureSuspendedException {
     // Local master locks don't store any state, so on recovery, simply finish this procedure
     // immediately.
     if (recoveredMasterLock) return null;
@@ -334,6 +335,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
     setState(ProcedureProtos.ProcedureState.RUNNABLE);
   }
 
+  @Override
   protected void toStringClassDetails(final StringBuilder builder) {
     super.toStringClassDetails(builder);
     if (regionInfos != null) {
@@ -350,6 +352,10 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
     builder.append(", type=").append(type);
   }
 
+  public LockType getType() {
+    return type;
+  }
+
   private LockInterface setupLock() throws IllegalArgumentException {
     if (regionInfos != null) {
       return setupRegionLock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 48a0b62..b0baf85 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -21,27 +21,34 @@ package org.apache.hadoop.hbase.master.procedure;
 import com.google.common.annotations.VisibleForTesting;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerName;
 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.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
 import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler;
+import org.apache.hadoop.hbase.procedure2.LockAndQueue;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.LockStatus;
 import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.LockAndQueue;
 import org.apache.hadoop.hbase.procedure2.ProcedureDeque;
-import org.apache.hadoop.hbase.util.AvlUtil.AvlKeyComparator;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.util.AvlUtil.AvlIterableList;
+import org.apache.hadoop.hbase.util.AvlUtil.AvlKeyComparator;
 import org.apache.hadoop.hbase.util.AvlUtil.AvlLinkedNode;
 import org.apache.hadoop.hbase.util.AvlUtil.AvlTree;
 import org.apache.hadoop.hbase.util.AvlUtil.AvlTreeIterator;
@@ -226,7 +233,111 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     return pollResult;
   }
 
-  @VisibleForTesting
+  private LockInfo createLockInfo(LockInfo.ResourceType resourceType,
+      String resourceName, LockAndQueue queue) {
+    LockInfo info = new LockInfo();
+
+    info.setResourceType(resourceType);
+    info.setResourceName(resourceName);
+
+    if (queue.hasExclusiveLock()) {
+      info.setLockType(LockInfo.LockType.EXCLUSIVE);
+
+      Procedure<?> exclusiveLockOwnerProcedure = queue.getExclusiveLockOwnerProcedure();
+      ProcedureInfo exclusiveLockOwnerProcedureInfo =
+          ProcedureUtil.convertToProcedureInfo(exclusiveLockOwnerProcedure);
+      info.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedureInfo);
+    } else if (queue.getSharedLockCount() > 0) {
+      info.setLockType(LockInfo.LockType.SHARED);
+      info.setSharedLockCount(queue.getSharedLockCount());
+    }
+
+    for (Procedure<?> procedure : queue) {
+      if (!(procedure instanceof LockProcedure)) {
+        continue;
+      }
+
+      LockProcedure lockProcedure = (LockProcedure)procedure;
+      LockInfo.WaitingProcedure waitingProcedure = new LockInfo.WaitingProcedure();
+
+      switch (lockProcedure.getType()) {
+      case EXCLUSIVE:
+        waitingProcedure.setLockType(LockInfo.LockType.EXCLUSIVE);
+        break;
+      case SHARED:
+        waitingProcedure.setLockType(LockInfo.LockType.SHARED);
+        break;
+      }
+
+      ProcedureInfo procedureInfo = ProcedureUtil.convertToProcedureInfo(lockProcedure);
+      waitingProcedure.setProcedure(procedureInfo);
+
+      info.addWaitingProcedure(waitingProcedure);
+    }
+
+    return info;
+  }
+
+  @Override
+  public List<LockInfo> listLocks() {
+    schedLock();
+
+    try {
+      List<LockInfo> lockInfos = new ArrayList<>();
+
+      for (Entry<ServerName, LockAndQueue> entry : locking.serverLocks
+          .entrySet()) {
+        String serverName = entry.getKey().getServerName();
+        LockAndQueue queue = entry.getValue();
+
+        if (queue.isLocked()) {
+          LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.SERVER,
+              serverName, queue);
+          lockInfos.add(lockInfo);
+        }
+      }
+
+      for (Entry<String, LockAndQueue> entry : locking.namespaceLocks
+          .entrySet()) {
+        String namespaceName = entry.getKey();
+        LockAndQueue queue = entry.getValue();
+
+        if (queue.isLocked()) {
+          LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.NAMESPACE,
+              namespaceName, queue);
+          lockInfos.add(lockInfo);
+        }
+      }
+
+      for (Entry<TableName, LockAndQueue> entry : locking.tableLocks
+          .entrySet()) {
+        String tableName = entry.getKey().getNameAsString();
+        LockAndQueue queue = entry.getValue();
+
+        if (queue.isLocked()) {
+          LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.TABLE,
+              tableName, queue);
+          lockInfos.add(lockInfo);
+        }
+      }
+
+      for (Entry<String, LockAndQueue> entry : locking.regionLocks.entrySet()) {
+        String regionName = entry.getKey();
+        LockAndQueue queue = entry.getValue();
+
+        if (queue.isLocked()) {
+          LockInfo lockInfo = createLockInfo(LockInfo.ResourceType.REGION,
+              regionName, queue);
+          lockInfos.add(lockInfo);
+        }
+      }
+
+      return lockInfos;
+    } finally {
+      schedUnlock();
+    }
+  }
+
   @Override
   public void clear() {
     schedLock();
@@ -390,6 +501,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       super(serverName, serverLock);
     }
 
+    @Override
     public boolean requireExclusiveLock(Procedure proc) {
       ServerProcedureInterface spi = (ServerProcedureInterface)proc;
       switch (spi.getServerOperationType()) {
@@ -437,6 +549,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return true;
     }
 
+    @Override
     public boolean requireExclusiveLock(Procedure proc) {
       return requireTableExclusiveLock((TableProcedureInterface)proc);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 c841e61..b686114 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
@@ -27,9 +27,10 @@
   import="java.util.Set"
   import="org.apache.hadoop.conf.Configuration"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
-  import="org.apache.hadoop.hbase.ProcedureInfo"
   import="org.apache.hadoop.hbase.master.HMaster"
   import="org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv"
+  import="org.apache.hadoop.hbase.ProcedureInfo"
+  import="org.apache.hadoop.hbase.procedure2.LockInfo"
   import="org.apache.hadoop.hbase.procedure2.ProcedureExecutor"
   import="org.apache.hadoop.hbase.procedure2.store.wal.ProcedureWALFile"
   import="org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore"
@@ -55,6 +56,8 @@
       return cmp < 0 ? -1 : cmp > 0 ? 1 : 0;
     }
   });
+
+  List<LockInfo> locks = master.listLocks();
 %>
 <!--[if IE]>
 <!DOCTYPE html>
@@ -62,15 +65,15 @@
 <?xml version="1.0" encoding="UTF-8" ?>
 <html xmlns="http://www.w3.org/1999/xhtml">
 <head>
-    <meta charset="utf-8">
+    <meta charset="utf-8" />
     <title>HBase Master Procedures: <%= master.getServerName() %></title>
-    <meta name="viewport" content="width=device-width, initial-scale=1.0">
-    <meta name="description" content="">
-    <meta name="author" content="">
+    <meta name="viewport" content="width=device-width, initial-scale=1.0" />
+    <meta name="description" content="" />
+    <meta name="author" content="" />
 
-    <link href="/static/css/bootstrap.min.css" rel="stylesheet">
-    <link href="/static/css/bootstrap-theme.min.css" rel="stylesheet">
-    <link href="/static/css/hbase.css" rel="stylesheet">
+    <link href="/static/css/bootstrap.min.css" rel="stylesheet" />
+    <link href="/static/css/bootstrap-theme.min.css" rel="stylesheet" />
+    <link href="/static/css/hbase.css" rel="stylesheet" />
   </head>
 <body>
 <div class="navbar  navbar-fixed-top navbar-default">
@@ -87,7 +90,7 @@
             <ul class="nav navbar-nav">
                 <li><a href="/master-status">Home</a></li>
                 <li><a href="/tablesDetailed.jsp">Table Details</a></li>
-                <li><a href="/procedures.jsp">Procedures</a></li>
+                <li><a href="/procedures.jsp">Procedures &amp; Locks</a></li>
                 <li><a href="/logs/">Local Logs</a></li>
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/dump">Debug Dump</a></li>
@@ -116,43 +119,42 @@
         <th>Last Update</th>
         <th>Errors</th>
     </tr>
-    <tr>
-      <% for (ProcedureInfo procInfo : procedures) { %>
+    <% for (ProcedureInfo procInfo : procedures) { %>
       <tr>
-        <td><%= procInfo.getProcId() %></a></td>
-        <td><%= procInfo.hasParentId() ? procInfo.getParentId() : "" %></a></td>
-        <td><%= escapeXml(procInfo.getProcState().toString()) %></a></td>
-        <td><%= escapeXml(procInfo.getProcOwner()) %></a></td>
-        <td><%= escapeXml(procInfo.getProcName()) %></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>
+        <td><%= procInfo.getProcId() %></td>
+        <td><%= procInfo.hasParentId() ? procInfo.getParentId() : "" %></td>
+        <td><%= escapeXml(procInfo.getProcState().toString()) %></td>
+        <td><%= escapeXml(procInfo.getProcOwner()) %></td>
+        <td><%= escapeXml(procInfo.getProcName()) %></td>
+        <td><%= new Date(procInfo.getSubmittedTime()) %></td>
+        <td><%= new Date(procInfo.getLastUpdate()) %></td>
+        <td><%= escapeXml(procInfo.isFailed() ? procInfo.getException().getMessage() : "") %></td>
       </tr>
     <% } %>
   </table>
 </div>
-<br>
+<br />
 <div class="container-fluid content">
   <div class="row">
     <div class="page-header">
       <h2>Procedure WAL State</h2>
     </div>
   </div>
-<div class="tabbable">
-  <ul class="nav nav-pills">
-    <li class="active">
-      <a href="#tab_WALFiles" data-toggle="tab">WAL files</a>
-    </li>
-    <li class="">
-      <a href="#tab_WALFilesCorrupted" data-toggle="tab">Corrupted WAL files</a>
-     </li>
-    <li class="">
-      <a href="#tab_WALRollTime" data-toggle="tab">WAL roll time</a>
-     </li>
-     <li class="">
-       <a href="#tab_SyncStats" data-toggle="tab">Sync stats</a>
-     </li>
-  </ul>
+  <div class="tabbable">
+    <ul class="nav nav-pills">
+      <li class="active">
+        <a href="#tab_WALFiles" data-toggle="tab">WAL files</a>
+      </li>
+      <li class="">
+        <a href="#tab_WALFilesCorrupted" data-toggle="tab">Corrupted WAL files</a>
+      </li>
+      <li class="">
+        <a href="#tab_WALRollTime" data-toggle="tab">WAL roll time</a>
+      </li>
+      <li class="">
+        <a href="#tab_SyncStats" data-toggle="tab">Sync stats</a>
+      </li>
+    </ul>
     <div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
       <div class="tab-pane active" id="tab_WALFiles">
         <% if (procedureWALFiles != null && procedureWALFiles.size() > 0) { %>
@@ -168,8 +170,8 @@
             <tr>
               <td> <%= pwf.getLogId() %></td>
               <td> <%= StringUtils.humanSize(pwf.getSize()) %> </td>
-              <td> <%= new Date(pwf.getTimestamp()) %></a></td>
-              <td> <%= escapeXml(pwf.toString()) %></t>
+              <td> <%= new Date(pwf.getTimestamp()) %> </td>
+              <td> <%= escapeXml(pwf.toString()) %> </td>
             </tr>
             <% } %>
           </table>
@@ -190,8 +192,8 @@
           <tr>
             <td> <%= cwf.getLogId() %></td>
             <td> <%= StringUtils.humanSize(cwf.getSize()) %> </td>
-            <td> <%= new Date(cwf.getTimestamp()) %></a></td>
-            <td> <%= escapeXml(cwf.toString()) %></t>
+            <td> <%= new Date(cwf.getTimestamp()) %> </td>
+            <td> <%= escapeXml(cwf.toString()) %> </td>
           </tr>
           <% } %>
           </table>
@@ -223,7 +225,7 @@
           <% for (int i = syncMetricsBuff.size() - 1; i >= 0; --i) { %>
           <%    WALProcedureStore.SyncMetrics syncMetrics = syncMetricsBuff.get(i); %>
           <tr>
-            <td> <%= new Date(syncMetrics.getTimestamp()) %></a></td>
+            <td> <%= new Date(syncMetrics.getTimestamp()) %></td>
             <td> <%= StringUtils.humanTimeDiff(syncMetrics.getSyncWaitMs()) %></td>
             <td> <%= syncMetrics.getSyncedEntries() %></td>
             <td> <%= StringUtils.humanSize(syncMetrics.getTotalSyncedBytes()) %></td>
@@ -235,6 +237,51 @@
       </div>
   </div>
 </div>
+<br />
+<div class="container-fluid content">
+  <div class="row">
+      <div class="page-header">
+          <h1>Locks</h1>
+      </div>
+  </div>
+  <% for (LockInfo lock : locks) { %>
+    <h2><%= lock.getResourceType() %>: <%= lock.getResourceName() %></h2>
+    <%
+      switch (lock.getLockType()) {
+      case EXCLUSIVE:
+    %>
+    <p>Lock type: EXCLUSIVE</p>
+    <p>Owner procedure ID: <%= lock.getExclusiveLockOwnerProcedure().getProcId() %></p>
+    <%
+        break;
+      case SHARED:
+    %>
+    <p>Lock type: SHARED</p>
+    <p>Number of shared locks: <%= lock.getSharedLockCount() %></p>
+    <%
+        break;
+      }
+
+      List<LockInfo.WaitingProcedure> waitingProcedures = lock.getWaitingProcedures();
+
+      if (!waitingProcedures.isEmpty()) {
+    %>
+	    <h3>Waiting procedures</h3>
+	    <table class="table table-striped" width="90%" >
+		    <tr>
+		      <th>Lock type</th>
+		      <th>Procedure ID</th>
+		    </tr>
+		    <% for (LockInfo.WaitingProcedure waitingProcedure : waitingProcedures) { %>
+		      <tr>
+	          <td><%= waitingProcedure.getLockType() %></td>
+	          <td><%= waitingProcedure.getProcedure().getProcId() %></td>
+		      </tr>
+		    <% } %>
+	    </table>
+    <% } %>
+  <% } %>
+</div>
 <script src="/static/js/jquery.min.js" type="text/javascript"></script>
 <script src="/static/js/bootstrap.min.js" type="text/javascript"></script>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp b/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp
index 90f639b..75f75fc 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp
@@ -94,7 +94,7 @@
             <ul class="nav navbar-nav">
                 <li><a href="/master-status">Home</a></li>
                 <li><a href="/tablesDetailed.jsp">Table Details</a></li>
-                <li><a href="/procedures.jsp">Procedures</a></li>
+                <li><a href="/procedures.jsp">Procedures &amp; Locks</a></li>
                 <li><a href="/logs/">Local Logs</a></li>
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/dump">Debug Dump</a></li>

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/resources/hbase-webapps/master/snapshotsStats.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/snapshotsStats.jsp b/hbase-server/src/main/resources/hbase-webapps/master/snapshotsStats.jsp
index aa9a17f..58f74f4 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/snapshotsStats.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/snapshotsStats.jsp
@@ -81,7 +81,7 @@
             <ul class="nav navbar-nav">
                 <li><a href="/master-status">Home</a></li>
                 <li><a href="/tablesDetailed.jsp">Table Details</a></li>
-                <li><a href="/procedures.jsp">Procedures</a></li>
+                <li><a href="/procedures.jsp">Procedures &amp; Locks</a></li>
                 <li><a href="/logs/">Local Logs</a></li>
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/dump">Debug Dump</a></li>

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index 0f8a289..0e1d1cf 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -158,7 +158,7 @@
             <ul class="nav navbar-nav">
                 <li><a href="/master-status">Home</a></li>
                 <li><a href="/tablesDetailed.jsp">Table Details</a></li>
-                <li><a href="/procedures.jsp">Procedures</a></li>
+                <li><a href="/procedures.jsp">Procedures &amp; Locks</a></li>
                 <li><a href="/logs/">Local Logs</a></li>
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/dump">Debug Dump</a></li>

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/resources/hbase-webapps/master/tablesDetailed.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/tablesDetailed.jsp b/hbase-server/src/main/resources/hbase-webapps/master/tablesDetailed.jsp
index d21be3e..a485e8b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/tablesDetailed.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/tablesDetailed.jsp
@@ -64,7 +64,7 @@
               <ul class="nav navbar-nav">
                   <li class="active"><a href="/master-status">Home</a></li>
                   <li><a href="/tablesDetailed.jsp">Table Details</a></li>
-                  <li><a href="/procedures.jsp">Procedures</a></li>
+                  <li><a href="/procedures.jsp">Procedures &amp; Locks</a></li>
                   <li><a href="/logs/">Local Logs</a></li>
                   <li><a href="/logLevel">Log Level</a></li>
                   <li><a href="/dump">Debug Dump</a></li>

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp b/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
index 6cd6c92..a2e6733 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
@@ -60,7 +60,7 @@
                     <ul class="nav navbar-nav">
                         <li><a href="/master-status">Home</a></li>
                         <li><a href="/tablesDetailed.jsp">Table Details</a></li>
-                        <li><a href="/procedures.jsp">Procedures</a></li>
+                        <li><a href="/procedures.jsp">Procedures &amp; Locks</a></li>
                         <li><a href="/logs/">Local Logs</a></li>
                         <li><a href="/logLevel">Log Level</a></li>
                         <li><a href="/dump">Debug Dump</a></li>

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 6b52e0c..3b80406 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -125,6 +126,8 @@ public class TestMasterObserver {
     private boolean postAbortProcedureCalled;
     private boolean preListProceduresCalled;
     private boolean postListProceduresCalled;
+    private boolean preListLocksCalled;
+    private boolean postListLocksCalled;
     private boolean preMoveCalled;
     private boolean postMoveCalled;
     private boolean preAssignCalled;
@@ -726,6 +729,25 @@ public class TestMasterObserver {
     }
 
     @Override
+    public void preListLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+      preListLocksCalled = true;
+    }
+
+    @Override
+    public void postListLocks(ObserverContext<MasterCoprocessorEnvironment> ctx, List<LockInfo> lockInfoList)
+        throws IOException {
+      postListLocksCalled = true;
+    }
+
+    public boolean wasListLocksCalled() {
+      return preListLocksCalled && postListLocksCalled;
+    }
+
+    public boolean wasPreListLocksCalledOnly() {
+      return preListLocksCalled && !postListLocksCalled;
+    }
+
+    @Override
     public void preMove(ObserverContext<MasterCoprocessorEnvironment> env,
         HRegionInfo region, ServerName srcServer, ServerName destServer)
     throws IOException {
@@ -2164,6 +2186,22 @@ public class TestMasterObserver {
       cp.wasListProceduresCalled());
   }
 
+  @Test (timeout=180000)
+  public void testListLocksOperation() throws Exception {
+    MiniHBaseCluster cluster = UTIL.getHBaseCluster();
+
+    HMaster master = cluster.getMaster();
+    MasterCoprocessorHost host = master.getMasterCoprocessorHost();
+    CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
+        CPMasterObserver.class.getName());
+    cp.resetStates();
+
+    master.listLocks();
+    assertTrue(
+      "Coprocessor should be called on list locks request",
+      cp.wasListLocksCalled());
+  }
+
   private void deleteTable(Admin admin, TableName tableName) throws Exception {
     // NOTE: We need a latch because admin is not sync,
     // so the postOp coprocessor method may be called after the admin operation returned.

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 683e9b3..ff6b88e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -39,16 +39,14 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.mockito.Mockito;
-
 import com.google.protobuf.Service;
 
 public class MockNoopMasterServices implements MasterServices, Server {
@@ -221,6 +219,11 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
+  public List<LockInfo> listLocks() throws IOException {
+    return null;
+  }
+
+  @Override
   public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
     return null;  //To change body of implemented methods use File | Settings | File Templates.
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
index 356c84f..e23c90a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
@@ -18,21 +18,24 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
+import static org.junit.Assert.*;
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
 import java.util.Arrays;
-
+import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.locking.LockProcedure;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.LockInfo.WaitingProcedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -899,5 +902,161 @@ public class TestMasterProcedureScheduler {
       sb.append(")");
     }
   }
+
+  private static LockProcedure createLockProcedure(LockProcedure.LockType lockType, long procId) throws Exception {
+    LockProcedure procedure = new LockProcedure();
+
+    Field typeField = LockProcedure.class.getDeclaredField("type");
+    typeField.setAccessible(true);
+    typeField.set(procedure, lockType);
+
+    Method setProcIdMethod = Procedure.class.getDeclaredMethod("setProcId", long.class);
+    setProcIdMethod.setAccessible(true);
+    setProcIdMethod.invoke(procedure, procId);
+
+    return procedure;
+  }
+
+  private static LockProcedure createExclusiveLockProcedure(long procId) throws Exception {
+    return createLockProcedure(LockProcedure.LockType.EXCLUSIVE, procId);
+  }
+
+  private static LockProcedure createSharedLockProcedure(long procId) throws Exception {
+    return createLockProcedure(LockProcedure.LockType.SHARED, procId);
+  }
+
+  private static void assertLockResource(LockInfo lock,
+      LockInfo.ResourceType resourceType, String resourceName)
+  {
+    assertEquals(resourceType, lock.getResourceType());
+    assertEquals(resourceName, lock.getResourceName());
+  }
+
+  private static void assertExclusiveLock(LockInfo lock, long procId)
+  {
+    assertEquals(LockInfo.LockType.EXCLUSIVE, lock.getLockType());
+    assertEquals(procId, lock.getExclusiveLockOwnerProcedure().getProcId());
+    assertEquals(0, lock.getSharedLockCount());
+  }
+
+  private static void assertSharedLock(LockInfo lock, int lockCount)
+  {
+    assertEquals(LockInfo.LockType.SHARED, lock.getLockType());
+    assertEquals(lockCount, lock.getSharedLockCount());
+  }
+
+  @Test
+  public void testListLocksServer() throws Exception {
+    LockProcedure procedure = createExclusiveLockProcedure(0);
+    queue.waitServerExclusiveLock(procedure, ServerName.valueOf("server1,1234,0"));
+
+    List<LockInfo> locks = queue.listLocks();
+    assertEquals(1, locks.size());
+
+    LockInfo serverLock = locks.get(0);
+    assertLockResource(serverLock, LockInfo.ResourceType.SERVER, "server1,1234,0");
+    assertExclusiveLock(serverLock, 0);
+    assertTrue(serverLock.getWaitingProcedures().isEmpty());
+  }
+
+  @Test
+  public void testListLocksNamespace() throws Exception {
+    LockProcedure procedure = createExclusiveLockProcedure(1);
+    queue.waitNamespaceExclusiveLock(procedure, "ns1");
+
+    List<LockInfo> locks = queue.listLocks();
+    assertEquals(2, locks.size());
+
+    LockInfo namespaceLock = locks.get(0);
+    assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns1");
+    assertExclusiveLock(namespaceLock, 1);
+    assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+
+    LockInfo tableLock = locks.get(1);
+    assertLockResource(tableLock, LockInfo.ResourceType.TABLE,
+        TableName.NAMESPACE_TABLE_NAME.getNameAsString());
+    assertSharedLock(tableLock, 1);
+    assertTrue(tableLock.getWaitingProcedures().isEmpty());
+  }
+
+  @Test
+  public void testListLocksTable() throws Exception {
+    LockProcedure procedure = createExclusiveLockProcedure(2);
+    queue.waitTableExclusiveLock(procedure, TableName.valueOf("ns2", "table2"));
+
+    List<LockInfo> locks = queue.listLocks();
+    assertEquals(2, locks.size());
+
+    LockInfo namespaceLock = locks.get(0);
+    assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns2");
+    assertSharedLock(namespaceLock, 1);
+    assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+
+    LockInfo tableLock = locks.get(1);
+    assertLockResource(tableLock, LockInfo.ResourceType.TABLE, "ns2:table2");
+    assertExclusiveLock(tableLock, 2);
+    assertTrue(tableLock.getWaitingProcedures().isEmpty());
+  }
+
+  @Test
+  public void testListLocksRegion() throws Exception {
+    LockProcedure procedure = createExclusiveLockProcedure(3);
+    HRegionInfo regionInfo = new HRegionInfo(TableName.valueOf("ns3", "table3"));
+
+    queue.waitRegion(procedure, regionInfo);
+
+    List<LockInfo> locks = queue.listLocks();
+    assertEquals(3, locks.size());
+
+    LockInfo namespaceLock = locks.get(0);
+    assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns3");
+    assertSharedLock(namespaceLock, 1);
+    assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+
+    LockInfo tableLock = locks.get(1);
+    assertLockResource(tableLock, LockInfo.ResourceType.TABLE, "ns3:table3");
+    assertSharedLock(tableLock, 1);
+    assertTrue(tableLock.getWaitingProcedures().isEmpty());
+
+    LockInfo regionLock = locks.get(2);
+    assertLockResource(regionLock, LockInfo.ResourceType.REGION, regionInfo.getEncodedName());
+    assertExclusiveLock(regionLock, 3);
+    assertTrue(regionLock.getWaitingProcedures().isEmpty());
+  }
+
+  @Test
+  public void testListLocksWaiting() throws Exception {
+    LockProcedure procedure1 = createExclusiveLockProcedure(1);
+    queue.waitTableExclusiveLock(procedure1, TableName.valueOf("ns4", "table4"));
+
+    LockProcedure procedure2 = createSharedLockProcedure(2);
+    queue.waitTableSharedLock(procedure2, TableName.valueOf("ns4", "table4"));
+
+    LockProcedure procedure3 = createExclusiveLockProcedure(3);
+    queue.waitTableExclusiveLock(procedure3, TableName.valueOf("ns4", "table4"));
+
+    List<LockInfo> locks = queue.listLocks();
+    assertEquals(2, locks.size());
+
+    LockInfo namespaceLock = locks.get(0);
+    assertLockResource(namespaceLock, LockInfo.ResourceType.NAMESPACE, "ns4");
+    assertSharedLock(namespaceLock, 1);
+    assertTrue(namespaceLock.getWaitingProcedures().isEmpty());
+
+    LockInfo tableLock = locks.get(1);
+    assertLockResource(tableLock, LockInfo.ResourceType.TABLE, "ns4:table4");
+    assertExclusiveLock(tableLock, 1);
+
+    List<WaitingProcedure> waitingProcedures = tableLock.getWaitingProcedures();
+    assertEquals(2, waitingProcedures.size());
+
+    WaitingProcedure waitingProcedure1 = waitingProcedures.get(0);
+    assertEquals(LockInfo.LockType.SHARED, waitingProcedure1.getLockType());
+    assertEquals(2, waitingProcedure1.getProcedure().getProcId());
+
+    WaitingProcedure waitingProcedure2 = waitingProcedures.get(1);
+    assertEquals(LockInfo.LockType.EXCLUSIVE, waitingProcedure2.getLockType());
+    assertEquals(3, waitingProcedure2.getProcedure().getProcId());
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
index f943ce4..c88c370 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
@@ -20,21 +20,24 @@ package org.apache.hadoop.hbase.protobuf;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureState;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
@@ -336,4 +339,40 @@ public class TestProtobufUtil {
     Cell newOffheapKV = org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toCell(cell);
     assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
   }
+
+  private static ProcedureInfo createProcedureInfo(long procId)
+  {
+    return new ProcedureInfo(procId, "java.lang.Object", null,
+        ProcedureState.RUNNABLE, -1, null, null, 0, 0, null);
+  }
+
+  private static void assertProcedureInfoEquals(ProcedureInfo expected,
+      ProcedureInfo result)
+  {
+    if (expected == result) {
+      return;
+    } else if (expected == null || result == null) {
+      fail();
+    }
+
+    assertEquals(expected.getProcId(), result.getProcId());
+  }
+
+  private static void assertLockInfoEquals(LockInfo expected, LockInfo result)
+  {
+    assertEquals(expected.getResourceType(), result.getResourceType());
+    assertEquals(expected.getResourceName(), result.getResourceName());
+    assertEquals(expected.getLockType(), result.getLockType());
+    assertProcedureInfoEquals(expected.getExclusiveLockOwnerProcedure(),
+        result.getExclusiveLockOwnerProcedure());
+    assertEquals(expected.getSharedLockCount(), result.getSharedLockCount());
+  }
+
+  private static void assertWaitingProcedureEquals(
+      LockInfo.WaitingProcedure expected, LockInfo.WaitingProcedure result)
+  {
+    assertEquals(expected.getLockType(), result.getLockType());
+    assertProcedureInfoEquals(expected.getProcedure(),
+        result.getProcedure());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
new file mode 100644
index 0000000..da7c7c4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.shaded.protobuf;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureState;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestProtobufUtil {
+  public TestProtobufUtil() {
+  }
+
+  private static ProcedureInfo createProcedureInfo(long procId)
+  {
+    return new ProcedureInfo(procId, "java.lang.Object", null,
+        ProcedureState.RUNNABLE, -1, null, null, 0, 0, null);
+  }
+
+  private static void assertProcedureInfoEquals(ProcedureInfo expected,
+      ProcedureInfo result)
+  {
+    if (expected == result) {
+      return;
+    } else if (expected == null || result == null) {
+      fail();
+    }
+
+    assertEquals(expected.getProcId(), result.getProcId());
+  }
+
+  private static void assertLockInfoEquals(LockInfo expected, LockInfo result)
+  {
+    assertEquals(expected.getResourceType(), result.getResourceType());
+    assertEquals(expected.getResourceName(), result.getResourceName());
+    assertEquals(expected.getLockType(), result.getLockType());
+    assertProcedureInfoEquals(expected.getExclusiveLockOwnerProcedure(),
+        result.getExclusiveLockOwnerProcedure());
+    assertEquals(expected.getSharedLockCount(), result.getSharedLockCount());
+  }
+
+  private static void assertWaitingProcedureEquals(
+      LockInfo.WaitingProcedure expected, LockInfo.WaitingProcedure result)
+  {
+    assertEquals(expected.getLockType(), result.getLockType());
+    assertProcedureInfoEquals(expected.getProcedure(),
+        result.getProcedure());
+  }
+
+  @Test
+  public void testServerLockInfo() {
+    LockInfo lock = new LockInfo();
+    lock.setResourceType(LockInfo.ResourceType.SERVER);
+    lock.setResourceName("server");
+    lock.setLockType(LockInfo.LockType.SHARED);
+    lock.setSharedLockCount(2);
+
+    LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
+    LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
+
+    assertLockInfoEquals(lock, lock2);
+  }
+
+  @Test
+  public void testNamespaceLockInfo() {
+    LockInfo lock = new LockInfo();
+    lock.setResourceType(LockInfo.ResourceType.NAMESPACE);
+    lock.setResourceName("ns");
+    lock.setLockType(LockInfo.LockType.EXCLUSIVE);
+    lock.setExclusiveLockOwnerProcedure(createProcedureInfo(2));
+
+    LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
+    LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
+
+    assertLockInfoEquals(lock, lock2);
+  }
+
+  @Test
+  public void testTableLockInfo() {
+    LockInfo lock = new LockInfo();
+    lock.setResourceType(LockInfo.ResourceType.TABLE);
+    lock.setResourceName("table");
+    lock.setLockType(LockInfo.LockType.SHARED);
+    lock.setSharedLockCount(2);
+
+    LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
+    LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
+
+    assertLockInfoEquals(lock, lock2);
+  }
+
+  @Test
+  public void testRegionLockInfo() {
+    LockInfo lock = new LockInfo();
+    lock.setResourceType(LockInfo.ResourceType.REGION);
+    lock.setResourceName("region");
+    lock.setLockType(LockInfo.LockType.EXCLUSIVE);
+    lock.setExclusiveLockOwnerProcedure(createProcedureInfo(2));
+
+    LockServiceProtos.LockInfo proto = ProtobufUtil.toProtoLockInfo(lock);
+    LockInfo lock2 = ProtobufUtil.toLockInfo(proto);
+
+    assertLockInfoEquals(lock, lock2);
+  }
+
+  @Test
+  public void testExclusiveWaitingLockInfo() {
+    LockInfo.WaitingProcedure waitingProcedure = new LockInfo.WaitingProcedure();
+    waitingProcedure.setLockType(LockInfo.LockType.EXCLUSIVE);
+    waitingProcedure.setProcedure(createProcedureInfo(1));
+
+    LockServiceProtos.WaitingProcedure proto = ProtobufUtil.toProtoWaitingProcedure(waitingProcedure);
+    LockInfo.WaitingProcedure waitingProcedure2 = ProtobufUtil.toWaitingProcedure(proto);
+
+    assertWaitingProcedureEquals(waitingProcedure, waitingProcedure2);
+  }
+
+  @Test
+  public void testSharedWaitingLockInfo() {
+    LockInfo.WaitingProcedure waitingProcedure = new LockInfo.WaitingProcedure();
+    waitingProcedure.setLockType(LockInfo.LockType.SHARED);
+    waitingProcedure.setProcedure(createProcedureInfo(2));
+
+    LockServiceProtos.WaitingProcedure proto = ProtobufUtil.toProtoWaitingProcedure(waitingProcedure);
+    LockInfo.WaitingProcedure waitingProcedure2 = ProtobufUtil.toWaitingProcedure(proto);
+
+    assertWaitingProcedureEquals(waitingProcedure, waitingProcedure2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 6aaa130..41904be 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1192,6 +1192,11 @@ module Hbase
       @admin.listProcedures()
     end
 
+    # List all locks
+    def list_locks()
+      @admin.listLocks();
+    end
+
     # Parse arguments and update HTableDescriptor accordingly
     def update_htd_from_arg(htd, arg)
       htd.setOwnerString(arg.delete(OWNER)) if arg[OWNER]

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 66480f9..fc55f94 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -436,10 +436,11 @@ Shell.load_command_group(
 
 Shell.load_command_group(
   'procedures',
-  :full_name => 'PROCEDURES MANAGEMENT',
+  :full_name => 'PROCEDURES & LOCKS MANAGEMENT',
   :commands => %w[
     abort_procedure
     list_procedures
+    list_locks
   ]
 )
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-shell/src/main/ruby/shell/commands.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands.rb b/hbase-shell/src/main/ruby/shell/commands.rb
index 271a7d9..08f2e11 100644
--- a/hbase-shell/src/main/ruby/shell/commands.rb
+++ b/hbase-shell/src/main/ruby/shell/commands.rb
@@ -98,6 +98,11 @@ module Shell
         @formatter ||= ::Shell::Formatter::Console.new
       end
 
+      # for testing purposes to catch the output of the commands
+      def set_formatter(formatter)
+        @formatter = formatter
+      end
+
       def translate_hbase_exceptions(*args)
         yield
       rescue => e

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-shell/src/main/ruby/shell/commands/list_locks.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_locks.rb b/hbase-shell/src/main/ruby/shell/commands/list_locks.rb
new file mode 100644
index 0000000..fca411b
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_locks.rb
@@ -0,0 +1,60 @@
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+module Shell
+  module Commands
+    class ListLocks < Command
+      def help
+        return <<-EOF
+List all locks in hbase. Examples:
+
+  hbase> list_locks
+EOF
+      end
+
+      def command()
+        list = admin.list_locks()
+
+        list.each do |lock|
+          formatter.output_strln("#{lock.resourceType}(#{lock.resourceName})")
+
+          case lock.lockType
+          when org.apache.hadoop.hbase.procedure2.LockInfo::LockType::EXCLUSIVE then
+            formatter.output_strln("Lock type: EXCLUSIVE, procedure: #{lock.exclusiveLockOwnerProcedure.procId}")
+          when org.apache.hadoop.hbase.procedure2.LockInfo::LockType::SHARED then
+            formatter.output_strln("Lock type: SHARED, count: #{lock.sharedLockCount}")
+          end
+
+          if lock.waitingProcedures.any?
+            formatter.output_strln("Waiting procedures:")
+            formatter.header([ "Lock type", "Procedure Id" ])
+
+            lock.waitingProcedures.each do |waitingProcedure|
+              formatter.row([ waitingProcedure.lockType.to_s, waitingProcedure.procedure.procId.to_s ]);
+            end
+
+            formatter.footer(lock.waitingProcedures.size)
+          end
+
+          formatter.output_strln("");
+        end
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-shell/src/main/ruby/shell/formatter.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/formatter.rb b/hbase-shell/src/main/ruby/shell/formatter.rb
index 2f800f6..aa81c86 100644
--- a/hbase-shell/src/main/ruby/shell/formatter.rb
+++ b/hbase-shell/src/main/ruby/shell/formatter.rb
@@ -17,6 +17,8 @@
 # limitations under the License.
 #
 
+require 'stringio'
+
 # Results formatter
 module Shell
   module Formatter
@@ -25,7 +27,7 @@ module Shell
       attr_reader :row_count
 
       def is_valid_io?(obj)
-        obj.instance_of?(IO) || obj == Kernel
+        obj.instance_of?(IO) || obj.instance_of?(StringIO) || obj == Kernel
       end
 
       def refresh_width()
@@ -166,6 +168,11 @@ module Shell
         output(@max_width, str)
       end
 
+      def output_strln(str)
+        output_str(str)
+        @out.puts
+      end
+
       def output(width, str)
         if str == nil
           str = ''

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-shell/src/test/ruby/shell/list_locks_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
new file mode 100644
index 0000000..fe132db
--- /dev/null
+++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
@@ -0,0 +1,152 @@
+#
+#
+# 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.
+#
+
+require 'hbase_constants'
+require 'shell'
+
+class ListLocksTest < Test::Unit::TestCase
+  def setup
+    @hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
+    @shell = Shell::Shell.new(@hbase)
+    @master = $TEST_CLUSTER.getHBaseClusterInterface.getMaster
+    @scheduler = @master.getMasterProcedureExecutor.getEnvironment.getProcedureScheduler
+
+    @string_io = StringIO.new
+
+    @list_locks = Shell::Commands::ListLocks.new(@shell)
+    @list_locks.set_formatter(Shell::Formatter::Base.new({ :output_stream => @string_io }))
+  end
+
+  def set_field(object, field_name, value)
+    field = object.getClass.getDeclaredField(field_name)
+    field.setAccessible(true)
+    field.set(object, value)
+  end
+
+  def create_lock(type, proc_id)
+    lock = org.apache.hadoop.hbase.master.locking.LockProcedure.new()
+    set_field(lock, "type", type)
+    lock.procId = proc_id
+
+    return lock
+  end
+
+  def create_exclusive_lock(proc_id)
+    return create_lock(org.apache.hadoop.hbase.master.locking.LockProcedure::LockType::EXCLUSIVE, proc_id)
+  end
+
+  def create_shared_lock(proc_id)
+    return create_lock(org.apache.hadoop.hbase.master.locking.LockProcedure::LockType::SHARED, proc_id)
+  end
+
+  define_test "list server locks" do
+    lock = create_exclusive_lock(0)
+
+    server_name = org.apache.hadoop.hbase.ServerName.valueOf("server1,1234,0")
+
+    @scheduler.waitServerExclusiveLock(lock, server_name)
+    @list_locks.command()
+    @scheduler.wakeServerExclusiveLock(lock, server_name)
+
+    assert_equal(
+      "SERVER(server1,1234,0)\n" <<
+      "Lock type: EXCLUSIVE, procedure: 0\n\n",
+      @string_io.string)
+  end
+
+  define_test "list namespace locks" do
+    lock = create_exclusive_lock(1)
+
+    @scheduler.waitNamespaceExclusiveLock(lock, "ns1")
+    @list_locks.command()
+    @scheduler.wakeNamespaceExclusiveLock(lock, "ns1")
+
+    assert_equal(
+      "NAMESPACE(ns1)\n" <<
+      "Lock type: EXCLUSIVE, procedure: 1\n\n" <<
+      "TABLE(hbase:namespace)\n" <<
+      "Lock type: SHARED, count: 1\n\n",
+      @string_io.string)
+  end
+
+  define_test "list table locks" do
+    lock = create_exclusive_lock(2)
+
+    table_name = org.apache.hadoop.hbase.TableName.valueOf("ns2", "table2")
+
+    @scheduler.waitTableExclusiveLock(lock, table_name)
+    @list_locks.command()
+    @scheduler.wakeTableExclusiveLock(lock, table_name)
+
+    assert_equal(
+      "NAMESPACE(ns2)\n" <<
+      "Lock type: SHARED, count: 1\n\n" <<
+      "TABLE(ns2:table2)\n" <<
+      "Lock type: EXCLUSIVE, procedure: 2\n\n",
+      @string_io.string)
+  end
+
+  define_test "list region locks" do
+    lock = create_exclusive_lock(3)
+
+    table_name = org.apache.hadoop.hbase.TableName.valueOf("ns3", "table3")
+    region_info = org.apache.hadoop.hbase.HRegionInfo.new(table_name)
+
+    @scheduler.waitRegion(lock, region_info)
+    @list_locks.command()
+    @scheduler.wakeRegion(lock, region_info)
+
+    assert_equal(
+      "NAMESPACE(ns3)\n" <<
+      "Lock type: SHARED, count: 1\n\n" <<
+      "TABLE(ns3:table3)\n" <<
+      "Lock type: SHARED, count: 1\n\n" <<
+      "REGION(" << region_info.getEncodedName << ")\n" <<
+      "Lock type: EXCLUSIVE, procedure: 3\n\n",
+      @string_io.string)
+  end
+
+  define_test "list waiting locks" do
+    table_name = org.apache.hadoop.hbase.TableName.valueOf("ns4", "table4")
+
+    lock1 = create_exclusive_lock(1)
+    set_field(lock1, "tableName", table_name)
+
+    lock2 = create_shared_lock(2)
+    set_field(lock2, "tableName", table_name)
+
+    @scheduler.waitTableExclusiveLock(lock1, table_name)
+    @scheduler.waitTableSharedLock(lock2, table_name)
+    @list_locks.command()
+    @scheduler.wakeTableExclusiveLock(lock1, table_name)
+    @scheduler.wakeTableSharedLock(lock2, table_name)
+
+    assert_equal(
+      "NAMESPACE(ns4)\n" <<
+      "Lock type: SHARED, count: 1\n\n" <<
+      "TABLE(ns4:table4)\n" <<
+      "Lock type: EXCLUSIVE, procedure: 1\n" <<
+      "Waiting procedures:\n" <<
+      "Lock type  Procedure Id\n" <<
+      " SHARED 2\n" <<
+      "1 row(s)\n\n",
+      @string_io.string)
+  end
+
+end


[04/40] hbase git commit: HBASE-17930 Avoid using Canary.sniff in HBaseTestingUtility

Posted by sy...@apache.org.
HBASE-17930 Avoid using Canary.sniff in HBaseTestingUtility


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

Branch: refs/heads/hbase-12439
Commit: 75d1e0361ac0f7e88ebb330f32b0e62cde997d0c
Parents: b35121d
Author: zhangduo <zh...@apache.org>
Authored: Mon Apr 17 17:26:23 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Apr 18 18:07:25 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/tool/Canary.java    | 46 +-------------------
 .../hadoop/hbase/HBaseTestingUtility.java       | 20 ++++++---
 2 files changed, 16 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/75d1e036/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index ee93cdb..9b048ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.ServerName;
 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.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -78,7 +79,6 @@ import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.Scan.ReadType;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.tool.Canary.RegionTask.TaskType;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -110,6 +110,7 @@ import org.apache.zookeeper.data.Stat;
  * 3. zookeeper mode - for each zookeeper instance, selects a zNode and
  * outputs some information about failure or latency.
  */
+@InterfaceAudience.Private
 public final class Canary implements Tool {
   // Sink interface used by the canary to outputs information
   public interface Sink {
@@ -1110,49 +1111,6 @@ public final class Canary implements Tool {
    * Canary entry point for specified table.
    * @throws Exception
    */
-  public static void sniff(final Admin admin, TableName tableName, boolean rawScanEnabled)
-      throws Exception {
-    sniff(admin, tableName, TaskType.READ, rawScanEnabled);
-  }
-  
-  /**
-   * Canary entry point for specified table.
-   * Keeping this method backward compatibility
-   * @throws Exception
-   */
-  public static void sniff(final Admin admin, TableName tableName)
-      throws Exception {
-    sniff(admin, tableName, TaskType.READ, false);
-  }
-
-  /**
-   * Canary entry point for specified table with task type(read/write)
-   * @throws Exception
-   */
-  public static void sniff(final Admin admin, TableName tableName, TaskType taskType,
-      boolean rawScanEnabled)   throws Exception {
-    List<Future<Void>> taskFutures =
-        Canary.sniff(admin, new StdOutSink(), tableName.getNameAsString(),
-          new ScheduledThreadPoolExecutor(1), taskType, rawScanEnabled);
-    for (Future<Void> future : taskFutures) {
-      future.get();
-    }
-  }
-  
-  /**
-   * Canary entry point for specified table with task type(read/write)
-   * Keeping this method backward compatible
-   * @throws Exception
-   */
-  public static void sniff(final Admin admin, TableName tableName, TaskType taskType)
-      throws Exception {
-    Canary.sniff(admin, tableName, taskType, false);
-  }
-
-  /**
-   * Canary entry point for specified table.
-   * @throws Exception
-   */
   private static List<Future<Void>> sniff(final Admin admin, final Sink sink, String tableName,
       ExecutorService executor, TaskType taskType, boolean rawScanEnabled) throws Exception {
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d1e036/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..acf2af0 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
@@ -70,7 +70,6 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
@@ -106,7 +105,6 @@ import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.VisibilityLabelsCache;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.tool.Canary;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -4045,10 +4043,20 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
       public boolean evaluate() throws IOException {
         boolean tableAvailable = getAdmin().isTableAvailable(tableName);
         if (tableAvailable) {
-          try {
-            Canary.sniff(getAdmin(), tableName);
-          } catch (Exception e) {
-            throw new IOException("Canary sniff failed for table " + tableName, e);
+          try (Table table = getConnection().getTable(tableName)) {
+            HTableDescriptor htd = table.getTableDescriptor();
+            for (HRegionLocation loc : getConnection().getRegionLocator(tableName)
+                .getAllRegionLocations()) {
+              Scan scan = new Scan().withStartRow(loc.getRegionInfo().getStartKey())
+                  .withStopRow(loc.getRegionInfo().getEndKey()).setOneRowLimit()
+                  .setMaxResultsPerColumnFamily(1).setCacheBlocks(false);
+              for (byte[] family : htd.getFamiliesKeys()) {
+                scan.addFamily(family);
+              }
+              try (ResultScanner scanner = table.getScanner(scan)) {
+                scanner.next();
+              }
+            }
           }
         }
         return tableAvailable;


[08/40] hbase git commit: HBASE-17914 Create a new reader instead of cloning a new StoreFile when compaction

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
index f2d00b3..b839fc3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
@@ -753,9 +753,6 @@ public class TestStripeCompactionPolicy {
     when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyLong(),
       anyBoolean())).thenReturn(mock(StoreFileScanner.class));
     when(sf.getReader()).thenReturn(r);
-    when(sf.createReader(anyBoolean())).thenReturn(r);
-    when(sf.createReader()).thenReturn(r);
-    when(sf.cloneForReader()).thenReturn(sf);
     return sf;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
index 54f310d..17ab004 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
@@ -153,7 +153,7 @@ public class TestHBaseFsckEncryption {
 
   private byte[] extractHFileKey(Path path) throws Exception {
     HFile.Reader reader = HFile.createReader(TEST_UTIL.getTestFileSystem(), path,
-      new CacheConfig(conf), conf);
+      new CacheConfig(conf), true, conf);
     try {
       reader.loadFileInfo();
       Encryption.Context cryptoContext = reader.getFileContext().getEncryptionContext();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala
index 795ce6d..d2b707e 100644
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala
+++ b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/BulkLoadSuite.scala
@@ -390,7 +390,7 @@ BeforeAndAfterEach with BeforeAndAfterAll  with Logging {
     val f1FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f1"))
     for ( i <- 0 until f1FileList.length) {
       val reader = HFile.createReader(fs, f1FileList(i).getPath,
-        new CacheConfig(config), config)
+        new CacheConfig(config), true, config)
       assert(reader.getCompressionAlgorithm.getName.equals("gz"))
       assert(reader.getDataBlockEncoding.name().equals("PREFIX"))
     }
@@ -400,7 +400,7 @@ BeforeAndAfterEach with BeforeAndAfterAll  with Logging {
     val f2FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f2"))
     for ( i <- 0 until f2FileList.length) {
       val reader = HFile.createReader(fs, f2FileList(i).getPath,
-        new CacheConfig(config), config)
+        new CacheConfig(config), true, config)
       assert(reader.getCompressionAlgorithm.getName.equals("none"))
       assert(reader.getDataBlockEncoding.name().equals("NONE"))
     }
@@ -869,7 +869,7 @@ BeforeAndAfterEach with BeforeAndAfterAll  with Logging {
     val f1FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f1"))
     for ( i <- 0 until f1FileList.length) {
       val reader = HFile.createReader(fs, f1FileList(i).getPath,
-        new CacheConfig(config), config)
+        new CacheConfig(config), true, config)
       assert(reader.getCompressionAlgorithm.getName.equals("gz"))
       assert(reader.getDataBlockEncoding.name().equals("PREFIX"))
     }
@@ -879,7 +879,7 @@ BeforeAndAfterEach with BeforeAndAfterAll  with Logging {
     val f2FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f2"))
     for ( i <- 0 until f2FileList.length) {
       val reader = HFile.createReader(fs, f2FileList(i).getPath,
-        new CacheConfig(config), config)
+        new CacheConfig(config), true, config)
       assert(reader.getCompressionAlgorithm.getName.equals("none"))
       assert(reader.getDataBlockEncoding.name().equals("NONE"))
     }


[34/40] hbase git commit: HBASE-15143 Procedure v2 - Web UI displaying queues

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 6dbf9b2..99853a5 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
@@ -104,6 +104,114 @@ public final class LockServiceProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.LockType)
   }
 
+  /**
+   * Protobuf enum {@code hbase.pb.ResourceType}
+   */
+  public enum ResourceType
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>RESOURCE_TYPE_SERVER = 1;</code>
+     */
+    RESOURCE_TYPE_SERVER(1),
+    /**
+     * <code>RESOURCE_TYPE_NAMESPACE = 2;</code>
+     */
+    RESOURCE_TYPE_NAMESPACE(2),
+    /**
+     * <code>RESOURCE_TYPE_TABLE = 3;</code>
+     */
+    RESOURCE_TYPE_TABLE(3),
+    /**
+     * <code>RESOURCE_TYPE_REGION = 4;</code>
+     */
+    RESOURCE_TYPE_REGION(4),
+    ;
+
+    /**
+     * <code>RESOURCE_TYPE_SERVER = 1;</code>
+     */
+    public static final int RESOURCE_TYPE_SERVER_VALUE = 1;
+    /**
+     * <code>RESOURCE_TYPE_NAMESPACE = 2;</code>
+     */
+    public static final int RESOURCE_TYPE_NAMESPACE_VALUE = 2;
+    /**
+     * <code>RESOURCE_TYPE_TABLE = 3;</code>
+     */
+    public static final int RESOURCE_TYPE_TABLE_VALUE = 3;
+    /**
+     * <code>RESOURCE_TYPE_REGION = 4;</code>
+     */
+    public static final int RESOURCE_TYPE_REGION_VALUE = 4;
+
+
+    public final int getNumber() {
+      return value;
+    }
+
+    /**
+     * @deprecated Use {@link #forNumber(int)} instead.
+     */
+    @java.lang.Deprecated
+    public static ResourceType valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static ResourceType forNumber(int value) {
+      switch (value) {
+        case 1: return RESOURCE_TYPE_SERVER;
+        case 2: return RESOURCE_TYPE_NAMESPACE;
+        case 3: return RESOURCE_TYPE_TABLE;
+        case 4: return RESOURCE_TYPE_REGION;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<ResourceType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        ResourceType> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<ResourceType>() {
+            public ResourceType findValueByNumber(int number) {
+              return ResourceType.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.protobuf.generated.LockServiceProtos.getDescriptor().getEnumTypes().get(1);
+    }
+
+    private static final ResourceType[] VALUES = values();
+
+    public static ResourceType 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 ResourceType(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.ResourceType)
+  }
+
   public interface LockRequestOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.LockRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
@@ -4898,70 +5006,2193 @@ public final class LockServiceProtos {
 
   }
 
+  public interface WaitingProcedureOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.WaitingProcedure)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 1;</code>
+     */
+    boolean hasLockType();
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType getLockType();
+
+    /**
+     * <code>required .hbase.pb.Procedure procedure = 2;</code>
+     */
+    boolean hasProcedure();
+    /**
+     * <code>required .hbase.pb.Procedure procedure = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure getProcedure();
+    /**
+     * <code>required .hbase.pb.Procedure procedure = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder getProcedureOrBuilder();
+  }
   /**
-   * Protobuf service {@code hbase.pb.LockService}
+   * Protobuf type {@code hbase.pb.WaitingProcedure}
    */
-  public static abstract class LockService
-      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.Service {
-    protected LockService() {}
+  public  static final class WaitingProcedure extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.WaitingProcedure)
+      WaitingProcedureOrBuilder {
+    // Use WaitingProcedure.newBuilder() to construct.
+    private WaitingProcedure(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private WaitingProcedure() {
+      lockType_ = 1;
+    }
 
-    public interface Interface {
-      /**
-       * <pre>
-       ** Acquire lock on namespace/table/region 
-       * </pre>
-       *
-       * <code>rpc RequestLock(.hbase.pb.LockRequest) returns (.hbase.pb.LockResponse);</code>
-       */
-      public abstract void requestLock(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest request,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse> done);
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private WaitingProcedure(
+        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: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType value = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                lockType_ = rawValue;
+              }
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = procedure_.toBuilder();
+              }
+              procedure_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(procedure_);
+                procedure_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              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.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_WaitingProcedure_descriptor;
+    }
 
-      /**
-       * <pre>
-       ** Keep alive (or not) a previously acquired lock 
-       * </pre>
-       *
-       * <code>rpc LockHeartbeat(.hbase.pb.LockHeartbeatRequest) returns (.hbase.pb.LockHeartbeatResponse);</code>
-       */
-      public abstract void lockHeartbeat(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest request,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse> done);
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_WaitingProcedure_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.class, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.Builder.class);
+    }
 
+    private int bitField0_;
+    public static final int LOCK_TYPE_FIELD_NUMBER = 1;
+    private int lockType_;
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 1;</code>
+     */
+    public boolean hasLockType() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType getLockType() {
+      org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType result = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.valueOf(lockType_);
+      return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.EXCLUSIVE : result;
     }
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
-        final Interface impl) {
-      return new LockService() {
-        @java.lang.Override
-        public  void requestLock(
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest request,
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse> done) {
-          impl.requestLock(controller, request, done);
-        }
+    public static final int PROCEDURE_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure procedure_;
+    /**
+     * <code>required .hbase.pb.Procedure procedure = 2;</code>
+     */
+    public boolean hasProcedure() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .hbase.pb.Procedure procedure = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure getProcedure() {
+      return procedure_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : procedure_;
+    }
+    /**
+     * <code>required .hbase.pb.Procedure procedure = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder getProcedureOrBuilder() {
+      return procedure_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : procedure_;
+    }
 
-        @java.lang.Override
-        public  void lockHeartbeat(
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest request,
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse> done) {
-          impl.lockHeartbeat(controller, request, done);
-        }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
 
-      };
+      if (!hasLockType()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasProcedure()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getProcedure().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
     }
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService
-        newReflectiveBlockingService(final BlockingInterface impl) {
-      return new org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingService() {
-        public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
-            getDescriptorForType() {
-          return getDescriptor();
-        }
-
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeEnum(1, lockType_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, getProcedure());
+      }
+      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
+          .computeEnumSize(1, lockType_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, getProcedure());
+      }
+      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.protobuf.generated.LockServiceProtos.WaitingProcedure)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure other = (org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure) obj;
+
+      boolean result = true;
+      result = result && (hasLockType() == other.hasLockType());
+      if (hasLockType()) {
+        result = result && lockType_ == other.lockType_;
+      }
+      result = result && (hasProcedure() == other.hasProcedure());
+      if (hasProcedure()) {
+        result = result && getProcedure()
+            .equals(other.getProcedure());
+      }
+      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 (hasLockType()) {
+        hash = (37 * hash) + LOCK_TYPE_FIELD_NUMBER;
+        hash = (53 * hash) + lockType_;
+      }
+      if (hasProcedure()) {
+        hash = (37 * hash) + PROCEDURE_FIELD_NUMBER;
+        hash = (53 * hash) + getProcedure().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure 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;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.WaitingProcedure}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.WaitingProcedure)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedureOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_WaitingProcedure_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_WaitingProcedure_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.class, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.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) {
+          getProcedureFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        lockType_ = 1;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (procedureBuilder_ == null) {
+          procedure_ = null;
+        } else {
+          procedureBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_WaitingProcedure_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure result = new org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.lockType_ = lockType_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (procedureBuilder_ == null) {
+          result.procedure_ = procedure_;
+        } else {
+          result.procedure_ = procedureBuilder_.build();
+        }
+        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.protobuf.generated.LockServiceProtos.WaitingProcedure) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.getDefaultInstance()) return this;
+        if (other.hasLockType()) {
+          setLockType(other.getLockType());
+        }
+        if (other.hasProcedure()) {
+          mergeProcedure(other.getProcedure());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasLockType()) {
+          return false;
+        }
+        if (!hasProcedure()) {
+          return false;
+        }
+        if (!getProcedure().isInitialized()) {
+          return false;
+        }
+        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.protobuf.generated.LockServiceProtos.WaitingProcedure 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.protobuf.generated.LockServiceProtos.WaitingProcedure) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private int lockType_ = 1;
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 1;</code>
+       */
+      public boolean hasLockType() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType getLockType() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType result = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.valueOf(lockType_);
+        return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.EXCLUSIVE : result;
+      }
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 1;</code>
+       */
+      public Builder setLockType(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        lockType_ = value.getNumber();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 1;</code>
+       */
+      public Builder clearLockType() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        lockType_ = 1;
+        onChanged();
+        return this;
+      }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure procedure_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> procedureBuilder_;
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public boolean hasProcedure() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure getProcedure() {
+        if (procedureBuilder_ == null) {
+          return procedure_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : procedure_;
+        } else {
+          return procedureBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public Builder setProcedure(org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure value) {
+        if (procedureBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          procedure_ = value;
+          onChanged();
+        } else {
+          procedureBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public Builder setProcedure(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder builderForValue) {
+        if (procedureBuilder_ == null) {
+          procedure_ = builderForValue.build();
+          onChanged();
+        } else {
+          procedureBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public Builder mergeProcedure(org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure value) {
+        if (procedureBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              procedure_ != null &&
+              procedure_ != org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance()) {
+            procedure_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.newBuilder(procedure_).mergeFrom(value).buildPartial();
+          } else {
+            procedure_ = value;
+          }
+          onChanged();
+        } else {
+          procedureBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public Builder clearProcedure() {
+        if (procedureBuilder_ == null) {
+          procedure_ = null;
+          onChanged();
+        } else {
+          procedureBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder getProcedureBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getProcedureFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder getProcedureOrBuilder() {
+        if (procedureBuilder_ != null) {
+          return procedureBuilder_.getMessageOrBuilder();
+        } else {
+          return procedure_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : procedure_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.Procedure procedure = 2;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> 
+          getProcedureFieldBuilder() {
+        if (procedureBuilder_ == null) {
+          procedureBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder>(
+                  getProcedure(),
+                  getParentForChildren(),
+                  isClean());
+          procedure_ = null;
+        }
+        return procedureBuilder_;
+      }
+      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:hbase.pb.WaitingProcedure)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.WaitingProcedure)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<WaitingProcedure>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<WaitingProcedure>() {
+      public WaitingProcedure 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 WaitingProcedure(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<WaitingProcedure> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<WaitingProcedure> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface LockInfoOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.LockInfo)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+     */
+    boolean hasResourceType();
+    /**
+     * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType getResourceType();
+
+    /**
+     * <code>optional string resource_name = 2;</code>
+     */
+    boolean hasResourceName();
+    /**
+     * <code>optional string resource_name = 2;</code>
+     */
+    java.lang.String getResourceName();
+    /**
+     * <code>optional string resource_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getResourceNameBytes();
+
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 3;</code>
+     */
+    boolean hasLockType();
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType getLockType();
+
+    /**
+     * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+     */
+    boolean hasExclusiveLockOwnerProcedure();
+    /**
+     * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure getExclusiveLockOwnerProcedure();
+    /**
+     * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder getExclusiveLockOwnerProcedureOrBuilder();
+
+    /**
+     * <code>optional int32 shared_lock_count = 5;</code>
+     */
+    boolean hasSharedLockCount();
+    /**
+     * <code>optional int32 shared_lock_count = 5;</code>
+     */
+    int getSharedLockCount();
+
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure> 
+        getWaitingProceduresList();
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure getWaitingProcedures(int index);
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    int getWaitingProceduresCount();
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedureOrBuilder> 
+        getWaitingProceduresOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedureOrBuilder getWaitingProceduresOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.LockInfo}
+   */
+  public  static final class LockInfo extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.LockInfo)
+      LockInfoOrBuilder {
+    // Use LockInfo.newBuilder() to construct.
+    private LockInfo(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private LockInfo() {
+      resourceType_ = 1;
+      resourceName_ = "";
+      lockType_ = 1;
+      sharedLockCount_ = 0;
+      waitingProcedures_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private LockInfo(
+        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: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType value = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                resourceType_ = rawValue;
+              }
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000002;
+              resourceName_ = bs;
+              break;
+            }
+            case 24: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType value = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(3, rawValue);
+              } else {
+                bitField0_ |= 0x00000004;
+                lockType_ = rawValue;
+              }
+              break;
+            }
+            case 34: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000008) == 0x00000008)) {
+                subBuilder = exclusiveLockOwnerProcedure_.toBuilder();
+              }
+              exclusiveLockOwnerProcedure_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(exclusiveLockOwnerProcedure_);
+                exclusiveLockOwnerProcedure_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000008;
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              sharedLockCount_ = input.readInt32();
+              break;
+            }
+            case 50: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+                waitingProcedures_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure>();
+                mutable_bitField0_ |= 0x00000020;
+              }
+              waitingProcedures_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure.PARSER, extensionRegistry));
+              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 {
+        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+          waitingProcedures_ = java.util.Collections.unmodifiableList(waitingProcedures_);
+        }
+        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.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_LockInfo_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_LockInfo_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.class, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int RESOURCE_TYPE_FIELD_NUMBER = 1;
+    private int resourceType_;
+    /**
+     * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+     */
+    public boolean hasResourceType() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType getResourceType() {
+      org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType result = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType.valueOf(resourceType_);
+      return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType.RESOURCE_TYPE_SERVER : result;
+    }
+
+    public static final int RESOURCE_NAME_FIELD_NUMBER = 2;
+    private volatile java.lang.Object resourceName_;
+    /**
+     * <code>optional string resource_name = 2;</code>
+     */
+    public boolean hasResourceName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string resource_name = 2;</code>
+     */
+    public java.lang.String getResourceName() {
+      java.lang.Object ref = resourceName_;
+      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()) {
+          resourceName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string resource_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getResourceNameBytes() {
+      java.lang.Object ref = resourceName_;
+      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);
+        resourceName_ = b;
+        return b;
+      } else {
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int LOCK_TYPE_FIELD_NUMBER = 3;
+    private int lockType_;
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 3;</code>
+     */
+    public boolean hasLockType() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required .hbase.pb.LockType lock_type = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType getLockType() {
+      org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType result = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.valueOf(lockType_);
+      return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.EXCLUSIVE : result;
+    }
+
+    public static final int EXCLUSIVE_LOCK_OWNER_PROCEDURE_FIELD_NUMBER = 4;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure exclusiveLockOwnerProcedure_;
+    /**
+     * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+     */
+    public boolean hasExclusiveLockOwnerProcedure() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure getExclusiveLockOwnerProcedure() {
+      return exclusiveLockOwnerProcedure_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : exclusiveLockOwnerProcedure_;
+    }
+    /**
+     * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder getExclusiveLockOwnerProcedureOrBuilder() {
+      return exclusiveLockOwnerProcedure_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : exclusiveLockOwnerProcedure_;
+    }
+
+    public static final int SHARED_LOCK_COUNT_FIELD_NUMBER = 5;
+    private int sharedLockCount_;
+    /**
+     * <code>optional int32 shared_lock_count = 5;</code>
+     */
+    public boolean hasSharedLockCount() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional int32 shared_lock_count = 5;</code>
+     */
+    public int getSharedLockCount() {
+      return sharedLockCount_;
+    }
+
+    public static final int WAITINGPROCEDURES_FIELD_NUMBER = 6;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure> waitingProcedures_;
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure> getWaitingProceduresList() {
+      return waitingProcedures_;
+    }
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedureOrBuilder> 
+        getWaitingProceduresOrBuilderList() {
+      return waitingProcedures_;
+    }
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    public int getWaitingProceduresCount() {
+      return waitingProcedures_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedure getWaitingProcedures(int index) {
+      return waitingProcedures_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.WaitingProcedure waitingProcedures = 6;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.WaitingProcedureOrBuilder getWaitingProceduresOrBuilder(
+        int index) {
+      return waitingProcedures_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (!hasResourceType()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLockType()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (hasExclusiveLockOwnerProcedure()) {
+        if (!getExclusiveLockOwnerProcedure().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getWaitingProceduresCount(); i++) {
+        if (!getWaitingProcedures(i).isInitialized()) {
+          memoizedIsInitialized = 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.writeEnum(1, resourceType_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, resourceName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeEnum(3, lockType_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeMessage(4, getExclusiveLockOwnerProcedure());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeInt32(5, sharedLockCount_);
+      }
+      for (int i = 0; i < waitingProcedures_.size(); i++) {
+        output.writeMessage(6, waitingProcedures_.get(i));
+      }
+      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
+          .computeEnumSize(1, resourceType_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, resourceName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeEnumSize(3, lockType_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, getExclusiveLockOwnerProcedure());
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeInt32Size(5, sharedLockCount_);
+      }
+      for (int i = 0; i < waitingProcedures_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(6, waitingProcedures_.get(i));
+      }
+      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.protobuf.generated.LockServiceProtos.LockInfo)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo other = (org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo) obj;
+
+      boolean result = true;
+      result = result && (hasResourceType() == other.hasResourceType());
+      if (hasResourceType()) {
+        result = result && resourceType_ == other.resourceType_;
+      }
+      result = result && (hasResourceName() == other.hasResourceName());
+      if (hasResourceName()) {
+        result = result && getResourceName()
+            .equals(other.getResourceName());
+      }
+      result = result && (hasLockType() == other.hasLockType());
+      if (hasLockType()) {
+        result = result && lockType_ == other.lockType_;
+      }
+      result = result && (hasExclusiveLockOwnerProcedure() == other.hasExclusiveLockOwnerProcedure());
+      if (hasExclusiveLockOwnerProcedure()) {
+        result = result && getExclusiveLockOwnerProcedure()
+            .equals(other.getExclusiveLockOwnerProcedure());
+      }
+      result = result && (hasSharedLockCount() == other.hasSharedLockCount());
+      if (hasSharedLockCount()) {
+        result = result && (getSharedLockCount()
+            == other.getSharedLockCount());
+      }
+      result = result && getWaitingProceduresList()
+          .equals(other.getWaitingProceduresList());
+      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 (hasResourceType()) {
+        hash = (37 * hash) + RESOURCE_TYPE_FIELD_NUMBER;
+        hash = (53 * hash) + resourceType_;
+      }
+      if (hasResourceName()) {
+        hash = (37 * hash) + RESOURCE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getResourceName().hashCode();
+      }
+      if (hasLockType()) {
+        hash = (37 * hash) + LOCK_TYPE_FIELD_NUMBER;
+        hash = (53 * hash) + lockType_;
+      }
+      if (hasExclusiveLockOwnerProcedure()) {
+        hash = (37 * hash) + EXCLUSIVE_LOCK_OWNER_PROCEDURE_FIELD_NUMBER;
+        hash = (53 * hash) + getExclusiveLockOwnerProcedure().hashCode();
+      }
+      if (hasSharedLockCount()) {
+        hash = (37 * hash) + SHARED_LOCK_COUNT_FIELD_NUMBER;
+        hash = (53 * hash) + getSharedLockCount();
+      }
+      if (getWaitingProceduresCount() > 0) {
+        hash = (37 * hash) + WAITINGPROCEDURES_FIELD_NUMBER;
+        hash = (53 * hash) + getWaitingProceduresList().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo 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;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.LockInfo}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.LockInfo)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfoOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_LockInfo_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_LockInfo_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.class, org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.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) {
+          getExclusiveLockOwnerProcedureFieldBuilder();
+          getWaitingProceduresFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        resourceType_ = 1;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        resourceName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        lockType_ = 1;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          exclusiveLockOwnerProcedure_ = null;
+        } else {
+          exclusiveLockOwnerProcedureBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        sharedLockCount_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        if (waitingProceduresBuilder_ == null) {
+          waitingProcedures_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000020);
+        } else {
+          waitingProceduresBuilder_.clear();
+        }
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.internal_static_hbase_pb_LockInfo_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo result = new org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.resourceType_ = resourceType_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.resourceName_ = resourceName_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.lockType_ = lockType_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          result.exclusiveLockOwnerProcedure_ = exclusiveLockOwnerProcedure_;
+        } else {
+          result.exclusiveLockOwnerProcedure_ = exclusiveLockOwnerProcedureBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.sharedLockCount_ = sharedLockCount_;
+        if (waitingProceduresBuilder_ == null) {
+          if (((bitField0_ & 0x00000020) == 0x00000020)) {
+            waitingProcedures_ = java.util.Collections.unmodifiableList(waitingProcedures_);
+            bitField0_ = (bitField0_ & ~0x00000020);
+          }
+          result.waitingProcedures_ = waitingProcedures_;
+        } else {
+          result.waitingProcedures_ = waitingProceduresBuilder_.build();
+        }
+        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.protobuf.generated.LockServiceProtos.LockInfo) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockInfo.getDefaultInstance()) return this;
+        if (other.hasResourceType()) {
+          setResourceType(other.getResourceType());
+        }
+        if (other.hasResourceName()) {
+          bitField0_ |= 0x00000002;
+          resourceName_ = other.resourceName_;
+          onChanged();
+        }
+        if (other.hasLockType()) {
+          setLockType(other.getLockType());
+        }
+        if (other.hasExclusiveLockOwnerProcedure()) {
+          mergeExclusiveLockOwnerProcedure(other.getExclusiveLockOwnerProcedure());
+        }
+        if (other.hasSharedLockCount()) {
+          setSharedLockCount(other.getSharedLockCount());
+        }
+        if (waitingProceduresBuilder_ == null) {
+          if (!other.waitingProcedures_.isEmpty()) {
+            if (waitingProcedures_.isEmpty()) {
+              waitingProcedures_ = other.waitingProcedures_;
+              bitField0_ = (bitField0_ & ~0x00000020);
+            } else {
+              ensureWaitingProceduresIsMutable();
+              waitingProcedures_.addAll(other.waitingProcedures_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.waitingProcedures_.isEmpty()) {
+            if (waitingProceduresBuilder_.isEmpty()) {
+              waitingProceduresBuilder_.dispose();
+              waitingProceduresBuilder_ = null;
+              waitingProcedures_ = other.waitingProcedures_;
+              bitField0_ = (bitField0_ & ~0x00000020);
+              waitingProceduresBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getWaitingProceduresFieldBuilder() : null;
+            } else {
+              waitingProceduresBuilder_.addAllMessages(other.waitingProcedures_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasResourceType()) {
+          return false;
+        }
+        if (!hasLockType()) {
+          return false;
+        }
+        if (hasExclusiveLockOwnerProcedure()) {
+          if (!getExclusiveLockOwnerProcedure().isInitialized()) {
+            return false;
+          }
+        }
+        for (int i = 0; i < getWaitingProceduresCount(); i++) {
+          if (!getWaitingProcedures(i).isInitialized()) {
+            return false;
+          }
+        }
+        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.protobuf.generated.LockServiceProtos.LockInfo 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.protobuf.generated.LockServiceProtos.LockInfo) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private int resourceType_ = 1;
+      /**
+       * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+       */
+      public boolean hasResourceType() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType getResourceType() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType result = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType.valueOf(resourceType_);
+        return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType.RESOURCE_TYPE_SERVER : result;
+      }
+      /**
+       * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+       */
+      public Builder setResourceType(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.ResourceType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        resourceType_ = value.getNumber();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.ResourceType resource_type = 1;</code>
+       */
+      public Builder clearResourceType() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        resourceType_ = 1;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object resourceName_ = "";
+      /**
+       * <code>optional string resource_name = 2;</code>
+       */
+      public boolean hasResourceName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string resource_name = 2;</code>
+       */
+      public java.lang.String getResourceName() {
+        java.lang.Object ref = resourceName_;
+        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()) {
+            resourceName_ = s;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string resource_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getResourceNameBytes() {
+        java.lang.Object ref = resourceName_;
+        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);
+          resourceName_ = b;
+          return b;
+        } else {
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string resource_name = 2;</code>
+       */
+      public Builder setResourceName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        resourceName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string resource_name = 2;</code>
+       */
+      public Builder clearResourceName() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        resourceName_ = getDefaultInstance().getResourceName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string resource_name = 2;</code>
+       */
+      public Builder setResourceNameBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        resourceName_ = value;
+        onChanged();
+        return this;
+      }
+
+      private int lockType_ = 1;
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 3;</code>
+       */
+      public boolean hasLockType() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType getLockType() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType result = org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.valueOf(lockType_);
+        return result == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType.EXCLUSIVE : result;
+      }
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 3;</code>
+       */
+      public Builder setLockType(org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000004;
+        lockType_ = value.getNumber();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.LockType lock_type = 3;</code>
+       */
+      public Builder clearLockType() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        lockType_ = 1;
+        onChanged();
+        return this;
+      }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure exclusiveLockOwnerProcedure_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> exclusiveLockOwnerProcedureBuilder_;
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public boolean hasExclusiveLockOwnerProcedure() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure getExclusiveLockOwnerProcedure() {
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          return exclusiveLockOwnerProcedure_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : exclusiveLockOwnerProcedure_;
+        } else {
+          return exclusiveLockOwnerProcedureBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public Builder setExclusiveLockOwnerProcedure(org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure value) {
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          exclusiveLockOwnerProcedure_ = value;
+          onChanged();
+        } else {
+          exclusiveLockOwnerProcedureBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public Builder setExclusiveLockOwnerProcedure(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder builderForValue) {
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          exclusiveLockOwnerProcedure_ = builderForValue.build();
+          onChanged();
+        } else {
+          exclusiveLockOwnerProcedureBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public Builder mergeExclusiveLockOwnerProcedure(org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure value) {
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              exclusiveLockOwnerProcedure_ != null &&
+              exclusiveLockOwnerProcedure_ != org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance()) {
+            exclusiveLockOwnerProcedure_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.newBuilder(exclusiveLockOwnerProcedure_).mergeFrom(value).buildPartial();
+          } else {
+            exclusiveLockOwnerProcedure_ = value;
+          }
+          onChanged();
+        } else {
+          exclusiveLockOwnerProcedureBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public Builder clearExclusiveLockOwnerProcedure() {
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          exclusiveLockOwnerProcedure_ = null;
+          onChanged();
+        } else {
+          exclusiveLockOwnerProcedureBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder getExclusiveLockOwnerProcedureBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getExclusiveLockOwnerProcedureFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder getExclusiveLockOwnerProcedureOrBuilder() {
+        if (exclusiveLockOwnerProcedureBuilder_ != null) {
+          return exclusiveLockOwnerProcedureBuilder_.getMessageOrBuilder();
+        } else {
+          return exclusiveLockOwnerProcedure_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance() : exclusiveLockOwnerProcedure_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.Procedure exclusive_lock_owner_procedure = 4;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder> 
+          getExclusiveLockOwnerProcedureFieldBuilder() {
+        if (exclusiveLockOwnerProcedureBuilder_ == null) {
+          exclusiveLockOwnerProcedureBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureOrBuilder>(
+                  getExclusiveLockOwnerProcedure(),
+                  getParentForChildren(),
+                  isClean());
+          exclusiveLockOwnerProcedure_ = null;
+        }
+        return exclusiveLockOwnerProcedureBuilder_;
+      }
+
+      private int sharedLockCount_ ;
+      /**
+       * <code>optional int32 shared_lock_count = 5;</code>
+       */
+      public boolean hasSharedLockCount() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional int32 shared_lock_count = 5;</code>
+       */
+      public int getSharedLockCount() {
+        return sharedLockCount_;
+      }
+      /**
+       * <code>optional int32 shared_lock_count = 5;</code>
+       */
+      public Builder setSharedLockCount(int value) {
+        bitField0_ |= 0x00000010;
+        sharedLockCount_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 shared_lock_count = 5;</code>
+       */
+      public Builder clearSharedL

<TRUNCATED>

[05/40] hbase git commit: HBASE-17936 Refine sum endpoint example in ref guide

Posted by sy...@apache.org.
HBASE-17936 Refine sum endpoint example in ref guide

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

Branch: refs/heads/hbase-12439
Commit: d15f75b3cfc5de4def04e94cbb965fd7f578dc34
Parents: 75d1e03
Author: Xiang Li <wa...@gmail.com>
Authored: Tue Apr 18 20:25:37 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Apr 18 09:33:09 2017 -0700

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/cp.adoc | 60 ++++++++++++++++++--------------
 1 file changed, 34 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d15f75b3/src/main/asciidoc/_chapters/cp.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc
index d3fcd47..d0dcfef 100644
--- a/src/main/asciidoc/_chapters/cp.adoc
+++ b/src/main/asciidoc/_chapters/cp.adoc
@@ -610,7 +610,7 @@ The effect is that the duplicate coprocessor is effectively ignored.
 +
 [source, java]
 ----
-public class SumEndPoint extends SumService implements Coprocessor, CoprocessorService {
+public class SumEndPoint extends Sum.SumService implements Coprocessor, CoprocessorService {
 
     private RegionCoprocessorEnvironment env;
 
@@ -630,31 +630,33 @@ public class SumEndPoint extends SumService implements Coprocessor, CoprocessorS
 
     @Override
     public void stop(CoprocessorEnvironment env) throws IOException {
-        // do mothing
+        // do nothing
     }
 
     @Override
-    public void getSum(RpcController controller, SumRequest request, RpcCallback done) {
+    public void getSum(RpcController controller, Sum.SumRequest request, RpcCallback<Sum.SumResponse> done) {
         Scan scan = new Scan();
         scan.addFamily(Bytes.toBytes(request.getFamily()));
         scan.addColumn(Bytes.toBytes(request.getFamily()), Bytes.toBytes(request.getColumn()));
-        SumResponse response = null;
+
+        Sum.SumResponse response = null;
         InternalScanner scanner = null;
+
         try {
             scanner = env.getRegion().getScanner(scan);
-            List results = new ArrayList();
+            List<Cell> results = new ArrayList<>();
             boolean hasMore = false;
-                        long sum = 0L;
-                do {
-                        hasMore = scanner.next(results);
-                        for (Cell cell : results) {
-                            sum = sum + Bytes.toLong(CellUtil.cloneValue(cell));
-                     }
-                        results.clear();
-                } while (hasMore);
+            long sum = 0L;
 
-                response = SumResponse.newBuilder().setSum(sum).build();
+            do {
+                hasMore = scanner.next(results);
+                for (Cell cell : results) {
+                    sum = sum + Bytes.toLong(CellUtil.cloneValue(cell));
+                }
+                results.clear();
+            } while (hasMore);
 
+            response = Sum.SumResponse.newBuilder().setSum(sum).build();
         } catch (IOException ioe) {
             ResponseConverter.setControllerException(controller, ioe);
         } finally {
@@ -664,6 +666,7 @@ public class SumEndPoint extends SumService implements Coprocessor, CoprocessorS
                 } catch (IOException ignored) {}
             }
         }
+
         done.run(response);
     }
 }
@@ -681,24 +684,29 @@ Table table = connection.getTable(tableName);
 //HConnection connection = HConnectionManager.createConnection(conf);
 //HTableInterface table = connection.getTable("users");
 
-final SumRequest request = SumRequest.newBuilder().setFamily("salaryDet").setColumn("gross")
-                            .build();
+final Sum.SumRequest request = Sum.SumRequest.newBuilder().setFamily("salaryDet").setColumn("gross").build();
 try {
-Map<byte[], Long> results = table.CoprocessorService (SumService.class, null, null,
-new Batch.Call<SumService, Long>() {
-    @Override
-        public Long call(SumService aggregate) throws IOException {
-BlockingRpcCallback rpcCallback = new BlockingRpcCallback();
-            aggregate.getSum(null, request, rpcCallback);
-            SumResponse response = rpcCallback.get();
-            return response.hasSum() ? response.getSum() : 0L;
+    Map<byte[], Long> results = table.coprocessorService(
+        Sum.SumService.class,
+        null,  /* start key */
+        null,  /* end   key */
+        new Batch.Call<Sum.SumService, Long>() {
+            @Override
+            public Long call(Sum.SumService aggregate) throws IOException {
+                BlockingRpcCallback<Sum.SumResponse> rpcCallback = new BlockingRpcCallback<>();
+                aggregate.getSum(null, request, rpcCallback);
+                Sum.SumResponse response = rpcCallback.get();
+
+                return response.hasSum() ? response.getSum() : 0L;
+            }
         }
-    });
+    );
+
     for (Long sum : results.values()) {
         System.out.println("Sum = " + sum);
     }
 } catch (ServiceException e) {
-e.printStackTrace();
+    e.printStackTrace();
 } catch (Throwable e) {
     e.printStackTrace();
 }


[15/40] hbase git commit: HBASE-17940 HMaster can not start due to Jasper related classes conflict

Posted by sy...@apache.org.
HBASE-17940 HMaster can not start due to Jasper related classes conflict


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

Branch: refs/heads/hbase-12439
Commit: 0953c144700c18b16f0d34de5ccec90e7c9cef3d
Parents: 3acd8e4
Author: zhangduo <zh...@apache.org>
Authored: Wed Apr 19 21:22:19 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Apr 20 16:06:50 2017 +0800

----------------------------------------------------------------------
 hbase-server/pom.xml |  9 ---------
 pom.xml              | 39 ---------------------------------------
 2 files changed, 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0953c144/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 10093cb..977a4c3 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -511,15 +511,6 @@
       <artifactId>zookeeper</artifactId>
     </dependency>
     <dependency>
-      <groupId>tomcat</groupId>
-      <artifactId>jasper-compiler</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>tomcat</groupId>
-      <artifactId>jasper-runtime</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.jamon</groupId>
       <artifactId>jamon-runtime</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/0953c144/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index aff01d6..c66c93c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1254,7 +1254,6 @@
     <metrics-core.version>3.1.2</metrics-core.version>
     <guava.version>12.0.1</guava.version>
     <jackson.version>2.23.2</jackson.version>
-    <jasper.version>5.5.23</jasper.version>
     <jaxb-api.version>2.2.2</jaxb-api.version>
     <jetty.version>9.3.8.v20160314</jetty.version>
     <jetty-jsp.version>9.2.19.v20160908</jetty-jsp.version>
@@ -1715,44 +1714,6 @@
         <version>${jackson1.version}</version>
       </dependency>
       <dependency>
-        <!--If this is not in the runtime lib, we get odd
-      "2009-02-27 11:38:39.504::WARN:  failed jsp
-       java.lang.NoSuchFieldError: IS_SECURITY_ENABLED"
-       exceptions out of jetty deploying webapps.
-       St.Ack Thu May 20 01:04:41 PDT 2010
-      -->
-        <groupId>tomcat</groupId>
-        <artifactId>jasper-compiler</artifactId>
-        <version>${jasper.version}</version>
-        <scope>runtime</scope>
-        <exclusions>
-          <exclusion>
-            <groupId>javax.servlet</groupId>
-            <artifactId>jsp-api</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>javax.servlet</groupId>
-            <artifactId>servlet-api</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>ant</groupId>
-            <artifactId>ant</artifactId>
-          </exclusion>
-        </exclusions>
-      </dependency>
-      <dependency>
-        <groupId>tomcat</groupId>
-        <artifactId>jasper-runtime</artifactId>
-        <version>${jasper.version}</version>
-        <scope>runtime</scope>
-        <exclusions>
-          <exclusion>
-            <groupId>javax.servlet</groupId>
-            <artifactId>servlet-api</artifactId>
-          </exclusion>
-        </exclusions>
-      </dependency>
-      <dependency>
         <groupId>org.jamon</groupId>
         <artifactId>jamon-runtime</artifactId>
         <version>${jamon-runtime.version}</version>


[37/40] hbase git commit: HBASE-15583 Any HTableDescriptor we give out should be immutable

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
new file mode 100644
index 0000000..c4ecacf
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
@@ -0,0 +1,376 @@
+/**
+ * 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.hadoop.hbase.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.BuilderStyleTest;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test setting values in the descriptor
+ */
+@Category({MiscTests.class, SmallTests.class})
+public class TestTableDescriptorBuilder {
+  private static final Log LOG = LogFactory.getLog(TestTableDescriptorBuilder.class);
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Test (expected=IOException.class)
+  public void testAddCoprocessorTwice() throws IOException {
+    String cpName = "a.b.c.d";
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+            .addCoprocessor(cpName)
+            .addCoprocessor(cpName)
+            .build();
+  }
+
+  @Test
+  public void testAddCoprocessorWithSpecStr() throws IOException {
+    String cpName = "a.b.c.d";
+    TableDescriptorBuilder builder
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME);
+
+    try {
+      builder.addCoprocessorWithSpec(cpName);
+      fail();
+    } catch (IllegalArgumentException iae) {
+      // Expected as cpName is invalid
+    }
+
+    // Try minimal spec.
+    try {
+      builder.addCoprocessorWithSpec("file:///some/path" + "|" + cpName);
+      fail();
+    } catch (IllegalArgumentException iae) {
+      // Expected to be invalid
+    }
+
+    // Try more spec.
+    String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
+    try {
+      builder.addCoprocessorWithSpec(spec);
+    } catch (IllegalArgumentException iae) {
+      fail();
+    }
+
+    // Try double add of same coprocessor
+    try {
+      builder.addCoprocessorWithSpec(spec);
+      fail();
+    } catch (IOException ioe) {
+      // Expect that the coprocessor already exists
+    }
+  }
+
+  @Test
+  public void testPb() throws DeserializationException, IOException {
+    final int v = 123;
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+          .setMaxFileSize(v)
+          .setDurability(Durability.ASYNC_WAL)
+          .setReadOnly(true)
+          .setRegionReplication(2)
+          .build();
+
+    byte [] bytes = TableDescriptorBuilder.toByteArray(htd);
+    TableDescriptor deserializedHtd = TableDescriptorBuilder.newBuilder(bytes).build();
+    assertEquals(htd, deserializedHtd);
+    assertEquals(v, deserializedHtd.getMaxFileSize());
+    assertTrue(deserializedHtd.isReadOnly());
+    assertEquals(Durability.ASYNC_WAL, deserializedHtd.getDurability());
+    assertEquals(deserializedHtd.getRegionReplication(), 2);
+  }
+
+  /**
+   * Test cps in the table description
+   * @throws Exception
+   */
+  @Test
+  public void testGetSetRemoveCP() throws Exception {
+    // simple CP
+    String className = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+         .addCoprocessor(className) // add and check that it is present
+        .build();
+    assertTrue(desc.hasCoprocessor(className));
+    desc = TableDescriptorBuilder.newBuilder(desc)
+         .removeCoprocessor(className) // remove it and check that it is gone
+        .build();
+    assertFalse(desc.hasCoprocessor(className));
+  }
+
+  /**
+   * Test cps in the table description
+   * @throws Exception
+   */
+  @Test
+  public void testSetListRemoveCP() throws Exception {
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
+    // Check that any coprocessor is present.
+    assertTrue(desc.getCoprocessors().isEmpty());
+
+    // simple CP
+    String className1 = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
+    String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            .addCoprocessor(className1) // Add the 1 coprocessor and check if present.
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertTrue(desc.getCoprocessors().contains(className1));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Add the 2nd coprocessor and check if present.
+            // remove it and check that it is gone
+            .addCoprocessor(className2)
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 2);
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Remove one and check
+            .removeCoprocessor(className1)
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Remove the last and check
+            .removeCoprocessor(className2)
+            .build();
+    assertTrue(desc.getCoprocessors().isEmpty());
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertFalse(desc.getCoprocessors().contains(className2));
+  }
+
+  /**
+   * Test that we add and remove strings from settings properly.
+   * @throws Exception
+   */
+  @Test
+  public void testRemoveString() throws Exception {
+    byte[] key = Bytes.toBytes("Some");
+    byte[] value = Bytes.toBytes("value");
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .setValue(key, value)
+            .build();
+    assertTrue(Bytes.equals(value, desc.getValue(key)));
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            .remove(key)
+            .build();
+    assertTrue(desc.getValue(key) == null);
+  }
+
+  String legalTableNames[] = { "foo", "with-dash_under.dot", "_under_start_ok",
+      "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02"
+      , "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
+      "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02"};
+  String illegalTableNames[] = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
+      "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
+      "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2"};
+
+  @Test
+  public void testLegalTableNames() {
+    for (String tn : legalTableNames) {
+      TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
+    }
+  }
+
+  @Test
+  public void testIllegalTableNames() {
+    for (String tn : illegalTableNames) {
+      try {
+        TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
+        fail("invalid tablename " + tn + " should have failed");
+      } catch (Exception e) {
+        // expected
+      }
+    }
+  }
+
+  @Test
+  public void testLegalTableNamesRegex() {
+    for (String tn : legalTableNames) {
+      TableName tName = TableName.valueOf(tn);
+      assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
+          tName.getNameAsString()));
+    }
+  }
+
+  @Test
+  public void testIllegalTableNamesRegex() {
+    for (String tn : illegalTableNames) {
+      LOG.info("Testing: '" + tn + "'");
+      assertFalse(Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tn));
+    }
+  }
+
+    /**
+   * Test default value handling for maxFileSize
+   */
+  @Test
+  public void testGetMaxFileSize() {
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    assertEquals(-1, desc.getMaxFileSize());
+    desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setMaxFileSize(1111L).build();
+    assertEquals(1111L, desc.getMaxFileSize());
+  }
+
+  /**
+   * Test default value handling for memStoreFlushSize
+   */
+  @Test
+  public void testGetMemStoreFlushSize() {
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    assertEquals(-1, desc.getMemStoreFlushSize());
+    desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setMemStoreFlushSize(1111L).build();
+    assertEquals(1111L, desc.getMemStoreFlushSize());
+  }
+
+  /**
+   * Test that we add and remove strings from configuration properly.
+   */
+  @Test
+  public void testAddGetRemoveConfiguration() {
+    String key = "Some";
+    String value = "value";
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setConfiguration(key, value)
+            .build();
+    assertEquals(value, desc.getConfigurationValue(key));
+    desc = TableDescriptorBuilder
+            .newBuilder(desc)
+            .removeConfiguration(key)
+            .build();
+    assertEquals(null, desc.getConfigurationValue(key));
+  }
+
+  @Test
+  public void testClassMethodsAreBuilderStyle() {
+    BuilderStyleTest.assertClassesAreBuilderStyle(TableDescriptorBuilder.class);
+  }
+
+  @Test
+  public void testModifyFamily() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(1000);
+    hcd.setDFSReplication((short) 3);
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+              .addFamily(hcd)
+              .build();
+
+    assertEquals(1000, htd.getFamily(familyName).getBlocksize());
+    assertEquals(3, htd.getFamily(familyName).getDFSReplication());
+    hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(2000);
+    hcd.setDFSReplication((short) 1);
+    htd = TableDescriptorBuilder.newBuilder(htd)
+              .modifyFamily(hcd)
+              .build();
+    assertEquals(2000, htd.getFamily(familyName).getBlocksize());
+    assertEquals(1, htd.getFamily(familyName).getDFSReplication());
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testModifyInexistentFamily() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    TableDescriptor htd = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .modifyFamily(hcd)
+            .build();
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testAddDuplicateFamilies() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(1000);
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcd)
+            .build();
+    assertEquals(1000, htd.getFamily(familyName).getBlocksize());
+    hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(2000);
+    // add duplicate column
+    TableDescriptorBuilder.newBuilder(htd).addFamily(hcd).build();
+  }
+
+  @Test
+  public void testPriority() {
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .setPriority(42)
+            .build();
+    assertEquals(42, htd.getPriority());
+  }
+
+  @Test
+  public void testSerialReplicationScope() {
+    HColumnDescriptor hcdWithScope = new HColumnDescriptor(Bytes.toBytes("cf0"));
+    hcdWithScope.setScope(HConstants.REPLICATION_SCOPE_SERIAL);
+    HColumnDescriptor hcdWithoutScope = new HColumnDescriptor(Bytes.toBytes("cf1"));
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithoutScope)
+            .build();
+    assertFalse(htd.hasSerialReplicationScope());
+
+    htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithScope)
+            .build();
+    assertTrue(htd.hasSerialReplicationScope());
+
+    htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithScope)
+            .addFamily(hcdWithoutScope)
+            .build();
+    assertTrue(htd.hasSerialReplicationScope());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
deleted file mode 100644
index dca0c1f..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.BuilderStyleTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ClientTests.class, SmallTests.class})
-public class TestUnmodifyableHTableDescriptor {
-
-    @Test
-    public void testClassMethodsAreBuilderStyle() {
-    /* UnmodifyableHTableDescriptor should have a builder style setup where setXXX/addXXX methods
-     * can be chainable together:
-     * . For example:
-     * UnmodifyableHTableDescriptor d
-     *   = new UnmodifyableHTableDescriptor()
-     *     .setFoo(foo)
-     *     .setBar(bar)
-     *     .setBuz(buz)
-     *
-     * This test ensures that all methods starting with "set" returns the declaring object
-     */
-
-        BuilderStyleTest.assertClassesAreBuilderStyle(UnmodifyableHTableDescriptor.class);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
index b4b00a9..406b4e5 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
@@ -111,7 +111,7 @@ public class RowResourceBase {
     if (admin.tableExists(TABLE_NAME)) {
       TEST_UTIL.deleteTable(TABLE_NAME);
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(CFA));
     htd.addFamily(new HColumnDescriptor(CFB));
     admin.createTable(htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
index 7c888e0..b926d82 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
@@ -32,6 +32,7 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.RestTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -116,7 +117,7 @@ public class TestRemoteAdminRetries {
     testTimedOutCall(new CallExecutor() {
       @Override
       public void run() throws Exception {
-        remoteAdmin.createTable(new HTableDescriptor(Bytes.toBytes("TestTable")));
+        remoteAdmin.createTable(new HTableDescriptor(TableName.valueOf("TestTable")));
       }
     });
     verify(client, times(RETRIES)).put(anyString(), anyString(), any(byte[].class));

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
index ab5c09f..e1fd82c 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
@@ -39,7 +39,6 @@ public interface RSGroupInfoManager {
   //Assigned before user tables
   TableName RSGROUP_TABLE_NAME =
       TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "rsgroup");
-  byte[] RSGROUP_TABLE_NAME_BYTES = RSGROUP_TABLE_NAME.toBytes();
   String rsGroupZNode = "rsgroup";
   byte[] META_FAMILY_BYTES = Bytes.toBytes("m");
   byte[] META_QUALIFIER_BYTES = Bytes.toBytes("i");

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 9f77c77..f2c6118 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -117,7 +117,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
   /** Table descriptor for <code>hbase:rsgroup</code> catalog table */
   private final static HTableDescriptor RSGROUP_TABLE_DESC;
   static {
-    RSGROUP_TABLE_DESC = new HTableDescriptor(RSGROUP_TABLE_NAME_BYTES);
+    RSGROUP_TABLE_DESC = new HTableDescriptor(RSGROUP_TABLE_NAME);
     RSGROUP_TABLE_DESC.addFamily(new HColumnDescriptor(META_FAMILY_BYTES));
     RSGROUP_TABLE_DESC.setRegionSplitPolicyClassName(DisabledRegionSplitPolicy.class.getName());
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 d6bd43b..6ef162b 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
@@ -189,7 +189,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     final byte[] tableName = Bytes.toBytes(tablePrefix + "_testCreateAndAssign");
     admin.modifyNamespace(NamespaceDescriptor.create("default")
         .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, "default").build());
-    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    final HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("f"));
     admin.createTable(desc);
     //wait for created table to be assigned

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 d34701f..f744ecb 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
@@ -176,7 +176,7 @@ public class RestoreTool {
         LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId);
 
         TableName newTableName = newTableNames[i];
-        HTableDescriptor newTableDescriptor = admin.getTableDescriptor(newTableName);
+        HTableDescriptor newTableDescriptor = new HTableDescriptor(admin.getTableDescriptor(newTableName));
         List<HColumnDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
         List<HColumnDescriptor> existingFamilies =
             Arrays.asList(newTableDescriptor.getColumnFamilies());
@@ -325,7 +325,7 @@ public class RestoreTool {
           LOG.debug("find table descriptor but no archive dir for table " + tableName
               + ", will only create table");
         }
-        tableDescriptor.setName(newTableName);
+        tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
         checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
           truncateIfExists);
         return;
@@ -338,7 +338,7 @@ public class RestoreTool {
     if (tableDescriptor == null) {
       tableDescriptor = new HTableDescriptor(newTableName);
     } else {
-      tableDescriptor.setName(newTableName);
+      tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
     }
 
     // record all region dirs:

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index c2ca3eb..70fe5c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableInfoMissingException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 
@@ -128,74 +129,69 @@ public class FSTableDescriptors implements TableDescriptors {
   @VisibleForTesting
   public static HTableDescriptor createMetaTableDescriptor(final Configuration conf)
       throws IOException {
-    HTableDescriptor metaDescriptor = new HTableDescriptor(
-        TableName.META_TABLE_NAME,
-        new HColumnDescriptor[] {
-            new HColumnDescriptor(HConstants.CATALOG_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+    return new HTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+          .addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                    // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_META_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_META_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.TABLE_FAMILY)
-                // Ten is arbitrary number.  Keep versions to help debugging.
-                .setMaxVersions(10)
-                .setInMemory(true)
-                .setBlocksize(8 * 1024)
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                    // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true)
-        }) {
-    };
-    metaDescriptor.addCoprocessor(
-        "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
-        null, Coprocessor.PRIORITY_SYSTEM, null);
-    return metaDescriptor;
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.TABLE_FAMILY)
+            // Ten is arbitrary number.  Keep versions to help debugging.
+            .setMaxVersions(10)
+            .setInMemory(true)
+            .setBlocksize(8 * 1024)
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addCoprocessor("org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
+            null, Coprocessor.PRIORITY_SYSTEM, null)
+          .build());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 c0ddbfc..e0edfa3 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
@@ -1726,7 +1726,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public static void setReplicas(Admin admin, TableName table, int replicaCount)
       throws IOException, InterruptedException {
     admin.disableTable(table);
-    HTableDescriptor desc = admin.getTableDescriptor(table);
+    HTableDescriptor desc = new HTableDescriptor(admin.getTableDescriptor(table));
     desc.setRegionReplication(replicaCount);
     admin.modifyTable(desc.getTableName(), desc);
     admin.enableTable(table);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
index 7744631..7457f43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
@@ -59,7 +59,7 @@ public class TestFSTableDescriptorForceCreation {
     // Cleanup old tests if any detritus laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     fstd.add(htd);
     assertFalse("Should not create new table descriptor", fstd.createTableDescriptor(htd, false));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index b7430fe..f6dbb41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -92,7 +92,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       TEST_UTIL.createTable(tables[i], FAMILY);
     }
 
-    HTableDescriptor[] tableDescs = admin.listTables().get();
+    TableDescriptor[] tableDescs = admin.listTables().get();
     int size = tableDescs.length;
     assertTrue(size >= tables.length);
     for (int i = 0; i < tables.length && i < size; i++) {
@@ -140,13 +140,13 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     htd.addFamily(fam2);
     htd.addFamily(fam3);
     admin.createTable(htd).join();
-    HTableDescriptor confirmedHtd = admin.getTableDescriptor(htd.getTableName()).get();
-    assertEquals(htd.compareTo(confirmedHtd), 0);
+    TableDescriptor confirmedHtd = admin.getTableDescriptor(htd.getTableName()).get();
+    assertEquals(htd.compareTo(new HTableDescriptor(confirmedHtd)), 0);
   }
 
   @Test(timeout = 300000)
   public void testCreateTable() throws Exception {
-    HTableDescriptor[] tables = admin.listTables().get();
+    TableDescriptor[] tables = admin.listTables().get();
     int numTables = tables.length;
     final  TableName tableName = TableName.valueOf(name.getMethodName());
     admin.createTable(new HTableDescriptor(tableName).addFamily(new HColumnDescriptor(FAMILY)))
@@ -452,7 +452,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
           } catch (Exception e) {
           }
         });
-    HTableDescriptor[] failed = admin.deleteTables(Pattern.compile("testDeleteTables.*")).get();
+    TableDescriptor[] failed = admin.deleteTables(Pattern.compile("testDeleteTables.*")).get();
     assertEquals(0, failed.length);
     Arrays.stream(tables).forEach((table) -> {
       admin.tableExists(table).thenAccept((exist) -> assertFalse(exist)).join();
@@ -727,7 +727,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       // Modify colymn family
       admin.modifyColumnFamily(tableName, cfDescriptor).join();
 
-      HTableDescriptor htd = admin.getTableDescriptor(tableName).get();
+      TableDescriptor htd = admin.getTableDescriptor(tableName).get();
       HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
       assertTrue(hcfd.getBlocksize() == newBlockSize);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
index 30b3b5b..61ff2be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
@@ -216,7 +216,7 @@ public class TestAsyncTableBatch {
   @Test
   public void testPartialSuccess() throws IOException, InterruptedException, ExecutionException {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLE_NAME));
     htd.addCoprocessor(ErrorInjectObserver.class.getName());
     admin.modifyTable(TABLE_NAME, htd);
     AsyncTableBase table = tableGetter.apply(TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
index 56f4141..f46354c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
@@ -77,7 +77,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test(timeout = 300000)
   public void disableNotFullReplication() throws Exception {
-    HTableDescriptor table = admin2.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("notReplicatedFamily");
     table.addFamily(f);
     admin1.disableTable(tableName);
@@ -125,7 +125,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test(timeout = 300000)
   public void testEnableReplicationWhenTableDescriptorIsNotSameInClusters() throws Exception {
-    HTableDescriptor table = admin2.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("newFamily");
     table.addFamily(f);
     admin2.disableTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 19df42a..70a78de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -140,7 +140,7 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLENAME));
     htd.setNormalizationEnabled(true);
     admin.modifyTable(TABLENAME, htd);
 
@@ -217,7 +217,7 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = admin.getTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
     htd.setNormalizationEnabled(true);
     admin.modifyTable(tableName, htd);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
index 3e7477d..4f6d4c2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
@@ -195,7 +195,7 @@ public class TestEncryptionKeyRotation {
     // Start the cluster back up
     TEST_UTIL.startMiniHBaseCluster(1, 1);
     // Verify the table can still be loaded
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
     // Double check that the store file keys can be unwrapped
     storeFilePaths = findStorefilePaths(htd.getTableName());
     assertTrue(storeFilePaths.size() > 0);
@@ -266,7 +266,7 @@ public class TestEncryptionKeyRotation {
     HColumnDescriptor hcd = htd.getFamilies().iterator().next();
     // Create the test table
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
     // Create a store file
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
index 3d8eeed..cd1f1e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
@@ -101,7 +101,7 @@ public class TestEncryptionRandomKeying {
 
     // Create the test table
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
 
     // Create a store file
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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..53f0892 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
@@ -5734,7 +5734,7 @@ public class TestHRegion {
       LOG.info("RegionEventDescriptor from WAL: " + desc);
 
       assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
-      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
       assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
         hri.getEncodedNameAsBytes()));
       assertTrue(desc.getLogSequenceNumber() > 0);
@@ -5854,7 +5854,7 @@ public class TestHRegion {
       LOG.info("RegionEventDescriptor from WAL: " + desc);
 
       assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
-      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
       assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
         hri.getEncodedNameAsBytes()));
       assertTrue(desc.getLogSequenceNumber() > 0);
@@ -5940,7 +5940,7 @@ public class TestHRegion {
     LOG.info("RegionEventDescriptor from WAL: " + desc);
 
     assertEquals(RegionEventDescriptor.EventType.REGION_CLOSE, desc.getEventType());
-    assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+    assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
     assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
       hri.getEncodedNameAsBytes()));
     assertTrue(desc.getLogSequenceNumber() > 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
index aeb909e..bc74f4f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
@@ -107,7 +107,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
     Connection connection = ConnectionFactory.createConnection(conf);
     Table t = connection.getTable(TEST_TABLE);
-    HTableDescriptor htd = t.getTableDescriptor();
+    HTableDescriptor htd = new HTableDescriptor(t.getTableDescriptor());
     htd.addCoprocessor("net.clayb.hbase.coprocessor.NotWhitelisted",
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);
@@ -153,7 +153,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     // coprocessor file
     admin.disableTable(TEST_TABLE);
     Table t = connection.getTable(TEST_TABLE);
-    HTableDescriptor htd = t.getTableDescriptor();
+    HTableDescriptor htd = new HTableDescriptor(t.getTableDescriptor());
     htd.addCoprocessor("net.clayb.hbase.coprocessor.Whitelisted",
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
index d301214..3e1abb9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
@@ -147,7 +147,7 @@ public class MobSnapshotTestingUtils {
 
     @Override
     public HTableDescriptor createHtd(final String tableName) {
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
       hcd.setMobEnabled(true);
       hcd.setMobThreshold(0L);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index ccad85b..3c10dee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -680,7 +680,7 @@ public final class SnapshotTestingUtils {
     }
 
     public HTableDescriptor createHtd(final String tableName) {
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
       return htd;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
index ee048b7..8337eb0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
@@ -261,7 +261,7 @@ public class TestFSTableDescriptors {
     final int count = 10;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       htds.createTableDescriptor(htd);
     }
 
@@ -294,7 +294,7 @@ public class TestFSTableDescriptors {
     final int count = 4;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       htds.createTableDescriptor(htd);
     }
     // add hbase:meta
@@ -321,7 +321,7 @@ public class TestFSTableDescriptors {
     final int count = 10;
     // Write out table infos via non-cached FSTableDescriptors
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       nonchtds.createTableDescriptor(htd);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
index 17ab004..c1d5778 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
@@ -92,7 +92,7 @@ public class TestHBaseFsckEncryption {
       cfKey));
     htd.addFamily(hcd);
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index 1d09dfa..74ef414 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -248,7 +248,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
 
-      HTableDescriptor htd = admin.getTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
       htd.setValue("NOT_DEFAULT", "true");
       admin.disableTable(tableName);
       admin.modifyTable(tableName, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 41904be..2db89b7 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -589,7 +589,7 @@ module Hbase
       table_name = TableName.valueOf(table_name_str)
 
       # Get table descriptor
-      htd = @admin.getTableDescriptor(table_name)
+      htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getTableDescriptor(table_name))
       hasTableUpdate = false
 
       # Process all args
@@ -1206,15 +1206,6 @@ module Hbase
       htd.setNormalizationEnabled(
         JBoolean.valueOf(arg.delete(NORMALIZATION_ENABLED))) if arg[NORMALIZATION_ENABLED]
       htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
-      # DEFERRED_LOG_FLUSH is deprecated and was replaced by DURABILITY.  To keep backward compatible, it still exists.
-      # However, it has to be set before DURABILITY so that DURABILITY could overwrite if both args are set
-      if arg.include?(DEFERRED_LOG_FLUSH)
-        if arg.delete(DEFERRED_LOG_FLUSH).to_s.upcase == "TRUE"
-          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("ASYNC_WAL"))
-        else
-          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("SYNC_WAL"))
-        end
-      end
       htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]
       htd.setPriority(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::PRIORITY))) if arg[org.apache.hadoop.hbase.HTableDescriptor::PRIORITY]
       htd.setFlushPolicyClassName(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::FLUSH_POLICY)) if arg[org.apache.hadoop.hbase.HTableDescriptor::FLUSH_POLICY]

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/alter_async.rb b/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
index e0f6deb..05335fb 100644
--- a/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
@@ -40,8 +40,8 @@ or a shorter version:
 
   hbase> alter_async 'ns1:t1', 'delete' => 'f1'
 
-You can also change table-scope attributes like MAX_FILESIZE
-MEMSTORE_FLUSHSIZE, READONLY, and DEFERRED_LOG_FLUSH.
+You can also change table-scope attributes like MAX_FILESIZE,
+MEMSTORE_FLUSHSIZE, and READONLY.
 
 For example, to change the max size of a family to 128MB, do:
 


[07/40] hbase git commit: HBASE-17925 mvn assembly:single fails against hadoop3-alpha2

Posted by sy...@apache.org.
HBASE-17925 mvn assembly:single fails against hadoop3-alpha2


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

Branch: refs/heads/hbase-12439
Commit: 719a30b11a7fddc5f8e5318064d8fb9ef5eab7cb
Parents: a8e6f33
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Fri Apr 14 13:45:07 2017 -0700
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Tue Apr 18 16:40:07 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/719a30b1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dcfd848..aff01d6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2381,7 +2381,7 @@
         <hadoop.version>${hadoop-three.version}</hadoop.version>
         <!--Use this compat module for now. TODO: Make h3 one if we need one-->
         <compat.module>hbase-hadoop2-compat</compat.module>
-        <assembly.file>src/main/assembly/hadoop-three-compat.xml</assembly.file>
+        <assembly.file>src/main/assembly/hadoop-two-compat.xml</assembly.file>
       </properties>
      <dependencyManagement>
        <dependencies>


[28/40] hbase git commit: HBASE-17514 emit a warning if thrift1 proxy user is configured but hbase.regionserver.thrift.http is not

Posted by sy...@apache.org.
HBASE-17514 emit a warning if thrift1 proxy user is configured but hbase.regionserver.thrift.http is not

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/9a1aff44
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9a1aff44
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9a1aff44

Branch: refs/heads/hbase-12439
Commit: 9a1aff447e908c9de351a4f45b869b016ad7821b
Parents: 435104a
Author: lv zehui <ma...@gmail.com>
Authored: Sat Apr 22 21:20:00 2017 +0800
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Apr 24 11:33:27 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9a1aff44/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
index 0829188..6a074fd 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -333,6 +333,11 @@ public class ThriftServerRunner implements Runnable {
     this.realUser = userProvider.getCurrent().getUGI();
     qop = conf.get(THRIFT_QOP_KEY);
     doAsEnabled = conf.getBoolean(THRIFT_SUPPORT_PROXYUSER, false);
+    if (doAsEnabled) {
+      if (!conf.getBoolean(USE_HTTP_CONF_KEY, false)) {
+        LOG.warn("Fail to enable the doAs feature. hbase.regionserver.thrift.http is not configured ");
+      }
+    }
     if (qop != null) {
       if (!qop.equals("auth") && !qop.equals("auth-int")
           && !qop.equals("auth-conf")) {


[31/40] hbase git commit: HBASE-17947 Location of Examples.proto is wrong in comment of RowCountEndPoint.java

Posted by sy...@apache.org.
HBASE-17947 Location of Examples.proto is wrong in comment of RowCountEndPoint.java

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/1367519c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1367519c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1367519c

Branch: refs/heads/hbase-12439
Commit: 1367519cd0545c2854108cffab03ae7c79b6ef2c
Parents: 72fac37
Author: Xiang Li <wa...@gmail.com>
Authored: Fri Apr 21 19:17:49 2017 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Apr 25 01:48:57 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1367519c/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java
index 36d8488..598008b 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java
@@ -45,7 +45,7 @@ import com.google.protobuf.Service;
  *
  * <p>
  * For the protocol buffer definition of the RowCountService, see the source file located under
- * hbase-server/src/main/protobuf/Examples.proto.
+ * hbase-examples/src/main/protobuf/Examples.proto.
  * </p>
  */
 public class RowCountEndpoint extends ExampleProtos.RowCountService


[35/40] hbase git commit: HBASE-15143 Procedure v2 - Web UI displaying queues

Posted by sy...@apache.org.
HBASE-15143 Procedure v2 - Web UI displaying queues

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

Branch: refs/heads/hbase-12439
Commit: 25575064154fe1cc7ff8970e8f15a3cff648f37a
Parents: 1367519
Author: Balazs Meszaros <ba...@cloudera.com>
Authored: Mon Feb 13 13:50:56 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Apr 25 09:39:28 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |    9 +
 .../hbase/client/ConnectionImplementation.java  |   11 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   42 +-
 .../client/ShortCircuitMasterConnection.java    |    6 +
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  183 +-
 .../hadoop/hbase/procedure2/LockInfo.java       |  128 +
 .../hadoop/hbase/procedure2/LockAndQueue.java   |   21 +-
 .../hadoop/hbase/procedure2/LockStatus.java     |    1 +
 .../hbase/procedure2/ProcedureScheduler.java    |    7 +
 .../hadoop/hbase/procedure2/ProcedureUtil.java  |    4 +-
 .../procedure2/SimpleProcedureScheduler.java    |   10 +-
 .../protobuf/generated/LockServiceProtos.java   | 2423 +++++++++++++++++-
 .../shaded/protobuf/generated/MasterProtos.java | 2152 ++++++++++++----
 .../src/main/protobuf/LockService.proto         |   22 +
 .../src/main/protobuf/Master.proto              |   11 +
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |    2 +-
 .../hbase/coprocessor/MasterObserver.java       |   19 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   37 +-
 .../hbase/master/MasterCoprocessorHost.java     |   21 +
 .../hadoop/hbase/master/MasterRpcServices.java  |  147 +-
 .../hadoop/hbase/master/MasterServices.java     |    9 +-
 .../hbase/master/locking/LockProcedure.java     |    8 +-
 .../procedure/MasterProcedureScheduler.java     |  119 +-
 .../hbase-webapps/master/procedures.jsp         |  127 +-
 .../resources/hbase-webapps/master/snapshot.jsp |    2 +-
 .../hbase-webapps/master/snapshotsStats.jsp     |    2 +-
 .../resources/hbase-webapps/master/table.jsp    |    2 +-
 .../hbase-webapps/master/tablesDetailed.jsp     |    2 +-
 .../main/resources/hbase-webapps/master/zk.jsp  |    2 +-
 .../hbase/coprocessor/TestMasterObserver.java   |   38 +
 .../hbase/master/MockNoopMasterServices.java    |    9 +-
 .../procedure/TestMasterProcedureScheduler.java |  169 +-
 .../hadoop/hbase/protobuf/TestProtobufUtil.java |   41 +-
 .../hbase/shaded/protobuf/TestProtobufUtil.java |  151 ++
 hbase-shell/src/main/ruby/hbase/admin.rb        |    5 +
 hbase-shell/src/main/ruby/shell.rb              |    3 +-
 hbase-shell/src/main/ruby/shell/commands.rb     |    5 +
 .../src/main/ruby/shell/commands/list_locks.rb  |   60 +
 hbase-shell/src/main/ruby/shell/formatter.rb    |    9 +-
 .../src/test/ruby/shell/list_locks_test.rb      |  152 ++
 40 files changed, 5409 insertions(+), 762 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 f2fc9a5..3e767d2 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
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
@@ -1250,6 +1251,14 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * List locks.
+   * @return lock list
+   * @throws IOException if a remote or network exception occurs
+   */
+  LockInfo[] listLocks()
+      throws IOException;
+
+  /**
    * Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.
    *
    * Note that the actual rolling of the log writer is asynchronous and may not be complete when

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 99feb14..6859cb3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -25,8 +25,6 @@ import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRI
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsentEx;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -120,6 +118,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import edu.umd.cs.findbugs.annotations.Nullable;
 
 /**
@@ -1283,6 +1283,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }
 
       @Override
+      public MasterProtos.ListLocksResponse listLocks(
+          RpcController controller,
+          MasterProtos.ListLocksRequest request) throws ServiceException {
+        return stub.listLocks(controller, request);
+      }
+
+      @Override
       public MasterProtos.AddColumnResponse addColumn(
           RpcController controller,
           MasterProtos.AddColumnRequest request) throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 e55a95d..7e79c20 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
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -80,6 +79,7 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
@@ -110,6 +110,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringP
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
@@ -151,6 +152,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedur
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListLocksResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
@@ -191,7 +194,6 @@ import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -201,7 +203,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.zookeeper.KeeperException;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
@@ -2096,26 +2097,33 @@ public class HBaseAdmin implements Admin {
             getRpcController(), ListProceduresRequest.newBuilder().build()).getProcedureList();
         ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
         for (int i = 0; i < procList.size(); i++) {
-          procInfoList[i] = convert(procList.get(i));
+          procInfoList[i] = ProtobufUtil.toProcedureInfo(procList.get(i));
         }
         return procInfoList;
       }
     });
   }
 
-  private static ProcedureInfo convert(final ProcedureProtos.Procedure procProto) {
-    NonceKey nonceKey = null;
-    if (procProto.getNonce() != HConstants.NO_NONCE) {
-      nonceKey = new NonceKey(procProto.getNonceGroup(), procProto.getNonce());
-    }
-    org.apache.hadoop.hbase.ProcedureState procedureState =
-        org.apache.hadoop.hbase.ProcedureState.valueOf(procProto.getState().name());
-    return new ProcedureInfo(procProto.getProcId(), procProto.getClassName(), procProto.getOwner(),
-        procedureState, procProto.hasParentId() ? procProto.getParentId() : -1, nonceKey,
-            procProto.hasException()?
-                ForeignExceptionUtil.toIOException(procProto.getException()): null,
-            procProto.getLastUpdate(), procProto.getSubmittedTime(),
-            procProto.hasResult()? procProto.getResult().toByteArray() : null);
+  @Override
+  public LockInfo[] listLocks() throws IOException {
+    return executeCallable(new MasterCallable<LockInfo[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected LockInfo[] rpcCall() throws Exception {
+        ListLocksRequest request = ListLocksRequest.newBuilder().build();
+        ListLocksResponse response = master.listLocks(getRpcController(), request);
+        List<LockServiceProtos.LockInfo> locksProto = response.getLockList();
+
+        LockInfo[] locks = new LockInfo[locksProto.size()];
+
+        for (int i = 0; i < locks.length; i++) {
+          LockServiceProtos.LockInfo lockProto = locksProto.get(i);
+          locks[i] = ProtobufUtil.toLockInfo(lockProto);
+        }
+
+        return locks;
+      }
+    });
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 72b2a15..e3b5b12 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
@@ -191,6 +191,12 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
   }
 
   @Override
+  public ListLocksResponse listLocks(RpcController controller,
+      ListLocksRequest request) throws ServiceException {
+    return stub.listLocks(controller, request);
+  }
+
+  @Override
   public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller,
       ListNamespaceDescriptorsRequest request) throws ServiceException {
     return stub.listNamespaceDescriptors(controller, request);

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index e969ded..04ce040 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -53,6 +53,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureState;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -82,6 +84,7 @@ import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.LimitInputStream;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.procedure2.LockInfo;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
 import org.apache.hadoop.hbase.quotas.QuotaType;
@@ -145,11 +148,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.Procedure;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
@@ -166,7 +172,9 @@ import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
+import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.ipc.RemoteException;
 
@@ -3262,4 +3270,177 @@ public final class ProtobufUtil {
     int port = Addressing.parsePort(str);
     return ServerName.valueOf(hostname, port, -1L);
   }
-}
\ No newline at end of file
+
+  /**
+   * @return Convert the current {@link ProcedureInfo} into a Protocol Buffers Procedure
+   * instance.
+   */
+  public static ProcedureProtos.Procedure toProtoProcedure(ProcedureInfo procedure) {
+    ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder();
+
+    builder.setClassName(procedure.getProcName());
+    builder.setProcId(procedure.getProcId());
+    builder.setSubmittedTime(procedure.getSubmittedTime());
+    builder.setState(ProcedureProtos.ProcedureState.valueOf(procedure.getProcState().name()));
+    builder.setLastUpdate(procedure.getLastUpdate());
+
+    if (procedure.hasParentId()) {
+      builder.setParentId(procedure.getParentId());
+    }
+
+    if (procedure.hasOwner()) {
+      builder.setOwner(procedure.getProcOwner());
+    }
+
+    if (procedure.isFailed()) {
+      builder.setException(ForeignExceptionUtil.toProtoForeignException(procedure.getException()));
+    }
+
+    if (procedure.hasResultData()) {
+      builder.setResult(UnsafeByteOperations.unsafeWrap(procedure.getResult()));
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Helper to convert the protobuf object.
+   * @return Convert the current Protocol Buffers Procedure to {@link ProcedureInfo}
+   * instance.
+   */
+  public static ProcedureInfo toProcedureInfo(ProcedureProtos.Procedure procedureProto) {
+    NonceKey nonceKey = null;
+
+    if (procedureProto.getNonce() != HConstants.NO_NONCE) {
+      nonceKey = new NonceKey(procedureProto.getNonceGroup(), procedureProto.getNonce());
+    }
+
+    return new ProcedureInfo(procedureProto.getProcId(), procedureProto.getClassName(),
+        procedureProto.hasOwner() ? procedureProto.getOwner() : null,
+        ProcedureState.valueOf(procedureProto.getState().name()),
+        procedureProto.hasParentId() ? procedureProto.getParentId() : -1, nonceKey,
+        procedureProto.hasException() ?
+          ForeignExceptionUtil.toIOException(procedureProto.getException()) : null,
+        procedureProto.getLastUpdate(), procedureProto.getSubmittedTime(),
+        procedureProto.hasResult() ? procedureProto.getResult().toByteArray() : null);
+  }
+
+  public static LockServiceProtos.ResourceType toProtoResourceType(
+      LockInfo.ResourceType resourceType) {
+    switch (resourceType) {
+    case SERVER:
+      return LockServiceProtos.ResourceType.RESOURCE_TYPE_SERVER;
+    case NAMESPACE:
+      return LockServiceProtos.ResourceType.RESOURCE_TYPE_NAMESPACE;
+    case TABLE:
+      return LockServiceProtos.ResourceType.RESOURCE_TYPE_TABLE;
+    case REGION:
+      return LockServiceProtos.ResourceType.RESOURCE_TYPE_REGION;
+    default:
+      throw new IllegalArgumentException("Unknown resource type: " + resourceType);
+    }
+  }
+
+  public static LockInfo.ResourceType toResourceType(
+      LockServiceProtos.ResourceType resourceTypeProto) {
+    switch (resourceTypeProto) {
+    case RESOURCE_TYPE_SERVER:
+      return LockInfo.ResourceType.SERVER;
+    case RESOURCE_TYPE_NAMESPACE:
+      return LockInfo.ResourceType.NAMESPACE;
+    case RESOURCE_TYPE_TABLE:
+      return LockInfo.ResourceType.TABLE;
+    case RESOURCE_TYPE_REGION:
+      return LockInfo.ResourceType.REGION;
+    default:
+      throw new IllegalArgumentException("Unknown resource type: " + resourceTypeProto);
+    }
+  }
+
+  public static LockServiceProtos.LockType toProtoLockType(
+      LockInfo.LockType lockType) {
+    return LockServiceProtos.LockType.valueOf(lockType.name());
+  }
+
+  public static LockInfo.LockType toLockType(
+      LockServiceProtos.LockType lockTypeProto) {
+    return LockInfo.LockType.valueOf(lockTypeProto.name());
+  }
+
+  public static LockServiceProtos.WaitingProcedure toProtoWaitingProcedure(
+      LockInfo.WaitingProcedure waitingProcedure) {
+    LockServiceProtos.WaitingProcedure.Builder builder = LockServiceProtos.WaitingProcedure.newBuilder();
+
+    ProcedureProtos.Procedure procedureProto =
+        toProtoProcedure(waitingProcedure.getProcedure());
+
+    builder
+        .setLockType(toProtoLockType(waitingProcedure.getLockType()))
+        .setProcedure(procedureProto);
+
+    return builder.build();
+  }
+
+  public static LockInfo.WaitingProcedure toWaitingProcedure(
+      LockServiceProtos.WaitingProcedure waitingProcedureProto) {
+    LockInfo.WaitingProcedure waiting = new LockInfo.WaitingProcedure();
+
+    waiting.setLockType(toLockType(waitingProcedureProto.getLockType()));
+
+    ProcedureInfo procedure =
+        toProcedureInfo(waitingProcedureProto.getProcedure());
+    waiting.setProcedure(procedure);
+
+    return waiting;
+  }
+
+  public static LockServiceProtos.LockInfo toProtoLockInfo(LockInfo lock)
+  {
+    LockServiceProtos.LockInfo.Builder builder = LockServiceProtos.LockInfo.newBuilder();
+
+    builder
+        .setResourceType(toProtoResourceType(lock.getResourceType()))
+        .setResourceName(lock.getResourceName())
+        .setLockType(toProtoLockType(lock.getLockType()));
+
+    ProcedureInfo exclusiveLockOwnerProcedure = lock.getExclusiveLockOwnerProcedure();
+
+    if (exclusiveLockOwnerProcedure != null) {
+      Procedure exclusiveLockOwnerProcedureProto =
+          toProtoProcedure(lock.getExclusiveLockOwnerProcedure());
+      builder.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedureProto);
+    }
+
+    builder.setSharedLockCount(lock.getSharedLockCount());
+
+    for (LockInfo.WaitingProcedure waitingProcedure : lock.getWaitingProcedures()) {
+      builder.addWaitingProcedures(toProtoWaitingProcedure(waitingProcedure));
+    }
+
+    return builder.build();
+  }
+
+  public static LockInfo toLockInfo(LockServiceProtos.LockInfo lockProto)
+  {
+    LockInfo lock = new LockInfo();
+
+    lock.setResourceType(toResourceType(lockProto.getResourceType()));
+    lock.setResourceName(lockProto.getResourceName());
+    lock.setLockType(toLockType(lockProto.getLockType()));
+
+    if (lockProto.hasExclusiveLockOwnerProcedure()) {
+      ProcedureInfo exclusiveLockOwnerProcedureProto =
+          toProcedureInfo(lockProto.getExclusiveLockOwnerProcedure());
+
+      lock.setExclusiveLockOwnerProcedure(exclusiveLockOwnerProcedureProto);
+    }
+
+    lock.setSharedLockCount(lockProto.getSharedLockCount());
+
+    for (LockServiceProtos.WaitingProcedure waitingProcedureProto : lockProto.getWaitingProceduresList()) {
+      lock.addWaitingProcedure(toWaitingProcedure(waitingProcedureProto));
+    }
+
+    return lock;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java
new file mode 100644
index 0000000..30ecee8
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/procedure2/LockInfo.java
@@ -0,0 +1,128 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Public
+public class LockInfo {
+  @InterfaceAudience.Public
+  public enum ResourceType {
+    SERVER, NAMESPACE, TABLE, REGION
+  }
+
+  @InterfaceAudience.Public
+  public enum LockType {
+    EXCLUSIVE, SHARED
+  }
+
+  @InterfaceAudience.Public
+  public static class WaitingProcedure {
+    private LockType lockType;
+    private ProcedureInfo procedure;
+
+    public WaitingProcedure() {
+    }
+
+    public LockType getLockType() {
+      return lockType;
+    }
+
+    public void setLockType(LockType lockType) {
+      this.lockType = lockType;
+    }
+
+    public ProcedureInfo getProcedure() {
+      return procedure;
+    }
+
+    public void setProcedure(ProcedureInfo procedure) {
+      this.procedure = procedure;
+    }
+  }
+
+  private ResourceType resourceType;
+  private String resourceName;
+  private LockType lockType;
+  private ProcedureInfo exclusiveLockOwnerProcedure;
+  private int sharedLockCount;
+  private final List<WaitingProcedure> waitingProcedures;
+
+  public LockInfo() {
+    waitingProcedures = new ArrayList<>();
+  }
+
+  public ResourceType getResourceType() {
+    return resourceType;
+  }
+
+  public void setResourceType(ResourceType resourceType) {
+    this.resourceType = resourceType;
+  }
+
+  public String getResourceName() {
+    return resourceName;
+  }
+
+  public void setResourceName(String resourceName) {
+    this.resourceName = resourceName;
+  }
+
+  public LockType getLockType() {
+    return lockType;
+  }
+
+  public void setLockType(LockType lockType) {
+    this.lockType = lockType;
+  }
+
+  public ProcedureInfo getExclusiveLockOwnerProcedure() {
+    return exclusiveLockOwnerProcedure;
+  }
+
+  public void setExclusiveLockOwnerProcedure(
+      ProcedureInfo exclusiveLockOwnerProcedure) {
+    this.exclusiveLockOwnerProcedure = exclusiveLockOwnerProcedure;
+  }
+
+  public int getSharedLockCount() {
+    return sharedLockCount;
+  }
+
+  public void setSharedLockCount(int sharedLockCount) {
+    this.sharedLockCount = sharedLockCount;
+  }
+
+  public List<WaitingProcedure> getWaitingProcedures() {
+    return waitingProcedures;
+  }
+
+  public void setWaitingProcedures(List<WaitingProcedure> waitingProcedures) {
+    this.waitingProcedures.clear();
+    this.waitingProcedures.addAll(waitingProcedures);
+  }
+
+  public void addWaitingProcedure(WaitingProcedure waitingProcedure) {
+    waitingProcedures.add(waitingProcedure);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
index e11c23c..2c307b7 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
@@ -43,7 +43,7 @@ package org.apache.hadoop.hbase.procedure2;
  * We do not use ReentrantReadWriteLock directly because of its high memory overhead.
  */
 public class LockAndQueue extends ProcedureDeque implements LockStatus {
-  private long exclusiveLockProcIdOwner = Long.MIN_VALUE;
+  private Procedure<?> exclusiveLockOwnerProcedure = null;
   private int sharedLock = 0;
 
   // ======================================================================
@@ -57,12 +57,12 @@ public class LockAndQueue extends ProcedureDeque implements LockStatus {
 
   @Override
   public boolean hasExclusiveLock() {
-    return this.exclusiveLockProcIdOwner != Long.MIN_VALUE;
+    return this.exclusiveLockOwnerProcedure != null;
   }
 
   @Override
   public boolean isLockOwner(long procId) {
-    return exclusiveLockProcIdOwner == procId;
+    return getExclusiveLockProcIdOwner() == procId;
   }
 
   @Override
@@ -76,8 +76,17 @@ public class LockAndQueue extends ProcedureDeque implements LockStatus {
   }
 
   @Override
+  public Procedure<?> getExclusiveLockOwnerProcedure() {
+    return exclusiveLockOwnerProcedure;
+  }
+
+  @Override
   public long getExclusiveLockProcIdOwner() {
-    return exclusiveLockProcIdOwner;
+    if (exclusiveLockOwnerProcedure == null) {
+      return Long.MIN_VALUE;
+    } else {
+      return exclusiveLockOwnerProcedure.getProcId();
+    }
   }
 
   @Override
@@ -101,7 +110,7 @@ public class LockAndQueue extends ProcedureDeque implements LockStatus {
 
   public boolean tryExclusiveLock(final Procedure proc) {
     if (isLocked()) return hasLockAccess(proc);
-    exclusiveLockProcIdOwner = proc.getProcId();
+    exclusiveLockOwnerProcedure = proc;
     return true;
   }
 
@@ -110,7 +119,7 @@ public class LockAndQueue extends ProcedureDeque implements LockStatus {
    */
   public boolean releaseExclusiveLock(final Procedure proc) {
     if (isLockOwner(proc.getProcId())) {
-      exclusiveLockProcIdOwner = Long.MIN_VALUE;
+      exclusiveLockOwnerProcedure = null;
       return true;
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockStatus.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockStatus.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockStatus.java
index 9f2aae7..f32ef76 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockStatus.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockStatus.java
@@ -29,6 +29,7 @@ public interface LockStatus {
   boolean isLockOwner(long procId);
   boolean hasParentLock(final Procedure proc);
   boolean hasLockAccess(final Procedure proc);
+  Procedure<?> getExclusiveLockOwnerProcedure();
   long getExclusiveLockProcIdOwner();
   int getSharedLockCount();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
index 617532b..b5295e7 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.procedure2;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -121,6 +122,12 @@ public interface ProcedureScheduler {
   boolean waitEvent(ProcedureEvent event, Procedure procedure);
 
   /**
+   * List lock queues.
+   * @return the locks
+   */
+  List<LockInfo> listLocks();
+
+  /**
    * Returns the number of elements in this queue.
    * @return the number of elements in this queue.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/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 b4222c7..7ce7568 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
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.procedure2;
 
-import com.google.common.base.Preconditions;
-
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Modifier;
@@ -33,6 +31,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.NonceKey;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Helper to convert to/from ProcedureProtos
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/25575064/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
index 788f4ff..176a900 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
@@ -18,10 +18,13 @@
 
 package org.apache.hadoop.hbase.procedure2;
 
-import com.google.common.annotations.VisibleForTesting;
+import java.util.Collections;
+import java.util.List;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Simple scheduler for procedures
  */
@@ -73,4 +76,9 @@ public class SimpleProcedureScheduler extends AbstractProcedureScheduler {
   @Override
   public void completionCleanup(Procedure proc) {
   }
+
+  @Override
+  public List<LockInfo> listLocks() {
+    return Collections.emptyList();
+  }
 }
\ No newline at end of file


[36/40] hbase git commit: HBASE-17956 Raw scan should ignore TTL

Posted by sy...@apache.org.
HBASE-17956 Raw scan should ignore TTL


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

Branch: refs/heads/hbase-12439
Commit: 8973582bc67a8a759310876e41a0348a1e26c89c
Parents: 2557506
Author: zhangduo <zh...@apache.org>
Authored: Tue Apr 25 14:02:10 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 26 15:03:53 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/StoreScanner.java |  2 +-
 .../client/TestScannersFromClientSide.java      | 24 ++++++++++++++++++++
 2 files changed, 25 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8973582b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 3bc6a0f..e42979e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -158,7 +158,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     this.scan = scan;
     this.columns = columns;
     this.now = EnvironmentEdgeManager.currentTime();
-    this.oldestUnexpiredTS = now - scanInfo.getTtl();
+    this.oldestUnexpiredTS = scan.isRaw() ? 0L : now - scanInfo.getTtl();
     this.minVersions = scanInfo.getMinVersions();
 
      // We look up row-column Bloom filters for multi-column queries as part of

http://git-wip-us.apache.org/repos/asf/hbase/blob/8973582b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index e5c19ac..1b18ee2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -16,9 +16,12 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -30,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -795,4 +799,24 @@ public class TestScannersFromClientSide {
 
     assertEquals(expKvList.size(), result.size());
   }
+
+  @Test
+  public void testReadExpiredDataForRawScan() throws IOException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    long ts = System.currentTimeMillis() - 10000;
+    byte[] value = Bytes.toBytes("expired");
+    try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
+      table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, ts, value));
+      assertArrayEquals(value, table.get(new Get(ROW)).getValue(FAMILY, QUALIFIER));
+      TEST_UTIL.getAdmin().modifyColumnFamily(tableName,
+        new HColumnDescriptor(FAMILY).setTimeToLive(5));
+      try (ResultScanner scanner = table.getScanner(FAMILY)) {
+        assertNull(scanner.next());
+      }
+      try (ResultScanner scanner = table.getScanner(new Scan().setRaw(true))) {
+        assertArrayEquals(value, scanner.next().getValue(FAMILY, QUALIFIER));
+        assertNull(scanner.next());
+      }
+    }
+  }
 }


[27/40] hbase git commit: HBASE-17302 The region flush request disappeared from flushQueue - addendum (Guangxu Cheng)

Posted by sy...@apache.org.
HBASE-17302 The region flush request disappeared from flushQueue - addendum (Guangxu Cheng)


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

Branch: refs/heads/hbase-12439
Commit: 435104af70232076145df4211da297c9235cd58f
Parents: 9053ec6
Author: tedyu <yu...@gmail.com>
Authored: Sun Apr 23 07:37:34 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sun Apr 23 07:37:34 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/435104af/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
index aaa9572..2f2a4cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
@@ -816,8 +816,8 @@ class MemStoreFlusher implements FlushRequester {
         return false;
       }
       FlushRegionEntry other = (FlushRegionEntry) obj;
-      if (!this.region.getRegionInfo().getRegionNameAsString().equals(
-              other.region.getRegionInfo().getRegionNameAsString())) {
+      if (!Bytes.equals(this.region.getRegionInfo().getRegionName(),
+          other.region.getRegionInfo().getRegionName())) {
         return false;
       }
       return compareTo(other) == 0;


[11/40] hbase git commit: HBASE-17929 addendum add scan read type option

Posted by sy...@apache.org.
HBASE-17929 addendum add scan read type option


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

Branch: refs/heads/hbase-12439
Commit: 6e962d6f65eebaa8b41e58368aa171a7d4b2910b
Parents: 66b616d
Author: zhangduo <zh...@apache.org>
Authored: Wed Apr 19 11:57:44 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 19 11:58:44 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/PerformanceEvaluation.java     | 22 ++++++++++++++------
 1 file changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6e962d6f/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 96ee515..f58c025 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -638,6 +638,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
             CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT);
     boolean asyncPrefetch = false;
     boolean cacheBlocks = true;
+    Scan.ReadType scanReadType = Scan.ReadType.DEFAULT;
 
     public TestOptions() {}
 
@@ -1248,9 +1249,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
     @Override
     void testRow(final int i) throws IOException {
-      Scan scan =
-          new Scan().withStartRow(getRandomRow(this.rand, opts.totalRows)).setCaching(opts.caching)
-              .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch);
+      Scan scan = new Scan().withStartRow(getRandomRow(this.rand, opts.totalRows))
+          .setCaching(opts.caching).setCacheBlocks(opts.cacheBlocks)
+          .setAsyncPrefetch(opts.asyncPrefetch).setReadType(opts.scanReadType);
       FilterList list = new FilterList();
       if (opts.addColumns) {
         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
@@ -1287,7 +1288,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
       Pair<byte[], byte[]> startAndStopRow = getStartAndStopRow();
       Scan scan = new Scan().withStartRow(startAndStopRow.getFirst())
           .withStopRow(startAndStopRow.getSecond()).setCaching(opts.caching)
-          .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch);
+          .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch)
+          .setReadType(opts.scanReadType);
       if (opts.filterAll) {
         scan.setFilter(new FilterAllFilter());
       }
@@ -1482,7 +1484,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       if (this.testScanner == null) {
         Scan scan = new Scan().withStartRow(format(opts.startRow)).setCaching(opts.caching)
-            .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch);
+            .setCacheBlocks(opts.cacheBlocks).setAsyncPrefetch(opts.asyncPrefetch)
+            .setReadType(opts.scanReadType);
         if (opts.addColumns) {
           scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
         } else {
@@ -1692,7 +1695,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         list.addFilter(new FilterAllFilter());
       }
       Scan scan = new Scan().setCaching(opts.caching).setCacheBlocks(opts.cacheBlocks)
-          .setAsyncPrefetch(opts.asyncPrefetch);
+          .setAsyncPrefetch(opts.asyncPrefetch).setReadType(opts.scanReadType);
       if (opts.addColumns) {
         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
       } else {
@@ -2170,6 +2173,13 @@ public class PerformanceEvaluation extends Configured implements Tool {
         opts.cacheBlocks = Boolean.parseBoolean(cmd.substring(cacheBlocks.length()));
         continue;
       }
+
+      final String scanReadType = "--scanReadType=";
+      if (cmd.startsWith(cacheBlocks)) {
+        opts.scanReadType =
+            Scan.ReadType.valueOf(cmd.substring(scanReadType.length()).toUpperCase());
+        continue;
+      }
       if (isCommandClass(cmd)) {
         opts.cmdName = cmd;
         try {


[39/40] hbase git commit: HBASE-15583 Any HTableDescriptor we give out should be immutable

Posted by sy...@apache.org.
HBASE-15583 Any HTableDescriptor we give out should be immutable


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

Branch: refs/heads/hbase-12439
Commit: 053e61541e6f45bbd2866faf4fe6c766a3f0c245
Parents: 8973582
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Apr 27 03:22:29 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Apr 27 03:22:29 2017 +0800

----------------------------------------------------------------------
 bin/region_status.rb                            |    2 +-
 .../apache/hadoop/hbase/HTableDescriptor.java   | 1055 ++---------
 .../org/apache/hadoop/hbase/client/Admin.java   |   26 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |   47 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   45 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |    7 +-
 .../org/apache/hadoop/hbase/client/HTable.java  |    2 +-
 .../hbase/client/ImmutableHTableDescriptor.java |   79 +
 .../hadoop/hbase/client/TableDescriptor.java    |  256 +++
 .../hbase/client/TableDescriptorBuilder.java    | 1639 ++++++++++++++++++
 .../client/UnmodifyableHTableDescriptor.java    |  127 --
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   46 +-
 .../hbase/shaded/protobuf/RequestConverter.java |    3 +-
 .../client/TestImmutableHTableDescriptor.java   |  102 ++
 .../client/TestTableDescriptorBuilder.java      |  376 ++++
 .../TestUnmodifyableHTableDescriptor.java       |   47 -
 .../hadoop/hbase/rest/RowResourceBase.java      |    2 +-
 .../rest/client/TestRemoteAdminRetries.java     |    3 +-
 .../hbase/rsgroup/RSGroupInfoManager.java       |    1 -
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |    2 +-
 .../hadoop/hbase/rsgroup/TestRSGroups.java      |    2 +-
 .../hadoop/hbase/backup/util/RestoreTool.java   |    6 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |  116 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |    2 +-
 .../TestFSTableDescriptorForceCreation.java     |    2 +-
 .../hbase/client/TestAsyncTableAdminApi.java    |   12 +-
 .../hbase/client/TestAsyncTableBatch.java       |    2 +-
 .../TestReplicationAdminWithClusters.java       |    4 +-
 .../TestSimpleRegionNormalizerOnCluster.java    |    4 +-
 .../regionserver/TestEncryptionKeyRotation.java |    4 +-
 .../TestEncryptionRandomKeying.java             |    2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |    6 +-
 .../TestCoprocessorWhitelistMasterObserver.java |    4 +-
 .../hbase/snapshot/MobSnapshotTestingUtils.java |    2 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |    2 +-
 .../hbase/util/TestFSTableDescriptors.java      |    6 +-
 .../hbase/util/TestHBaseFsckEncryption.java     |    2 +-
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |    2 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |   11 +-
 .../src/main/ruby/shell/commands/alter_async.rb |    4 +-
 40 files changed, 2820 insertions(+), 1242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/bin/region_status.rb
----------------------------------------------------------------------
diff --git a/bin/region_status.rb b/bin/region_status.rb
index 91873cb..f889de9 100644
--- a/bin/region_status.rb
+++ b/bin/region_status.rb
@@ -133,7 +133,7 @@ end
 
 # query the master to see how many regions are on region servers
 if not $tablename.nil?
-  $TableName = HTableDescriptor.new($tablename.to_java_bytes).getTableName()
+  $TableName = TableName.valueOf($tablename.to_java_bytes)
 end
 while true
   if $tablename.nil?

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 ed0659c..e3cf2ec 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
@@ -19,29 +19,20 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.regex.Matcher;
 
-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.client.Durability;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -49,271 +40,34 @@ import org.apache.hadoop.hbase.util.Bytes;
  * all the column families, is the table a catalog table, <code> -ROOT- </code> or
  * <code> hbase:meta </code>, if the table is read only, the maximum size of the memstore,
  * when the region split should occur, coprocessors associated with it etc...
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+ *             use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
  */
+@Deprecated
 @InterfaceAudience.Public
-public class HTableDescriptor implements Comparable<HTableDescriptor> {
-
-  private static final Log LOG = LogFactory.getLog(HTableDescriptor.class);
-
-  private TableName name = null;
-
-  /**
-   * A map which holds the metadata information of the table. This metadata
-   * includes values like IS_ROOT, IS_META, DEFERRED_LOG_FLUSH, SPLIT_POLICY,
-   * MAX_FILE_SIZE, READONLY, MEMSTORE_FLUSHSIZE etc...
-   */
-  private final Map<Bytes, Bytes> values = new HashMap<>();
-
-  /**
-   * A map which holds the configuration specific to the table.
-   * The keys of the map have the same names as config keys and override the defaults with
-   * table-specific settings. Example usage may be for compactions, etc.
-   */
-  private final Map<String, String> configuration = new HashMap<>();
-
-  public static final String SPLIT_POLICY = "SPLIT_POLICY";
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which denotes the maximum size of the store file after which
-   * a region split occurs
-   *
-   * @see #getMaxFileSize()
-   */
-  public static final String MAX_FILESIZE = "MAX_FILESIZE";
-  private static final Bytes MAX_FILESIZE_KEY =
-      new Bytes(Bytes.toBytes(MAX_FILESIZE));
-
-  public static final String OWNER = "OWNER";
-  public static final Bytes OWNER_KEY =
-      new Bytes(Bytes.toBytes(OWNER));
-
-  /**
-   * <em>INTERNAL</em> Used by rest interface to access this metadata
-   * attribute which denotes if the table is Read Only
-   *
-   * @see #isReadOnly()
-   */
-  public static final String READONLY = "READONLY";
-  private static final Bytes READONLY_KEY =
-      new Bytes(Bytes.toBytes(READONLY));
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which denotes if the table is compaction enabled
-   *
-   * @see #isCompactionEnabled()
-   */
-  public static final String COMPACTION_ENABLED = "COMPACTION_ENABLED";
-  private static final Bytes COMPACTION_ENABLED_KEY =
-      new Bytes(Bytes.toBytes(COMPACTION_ENABLED));
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which represents the maximum size of the memstore after which
-   * its contents are flushed onto the disk
-   *
-   * @see #getMemStoreFlushSize()
-   */
-  public static final String MEMSTORE_FLUSHSIZE = "MEMSTORE_FLUSHSIZE";
-  private static final Bytes MEMSTORE_FLUSHSIZE_KEY =
-      new Bytes(Bytes.toBytes(MEMSTORE_FLUSHSIZE));
-
-  public static final String FLUSH_POLICY = "FLUSH_POLICY";
-
-  /**
-   * <em>INTERNAL</em> Used by rest interface to access this metadata
-   * attribute which denotes if the table is a -ROOT- region or not
-   *
-   * @see #isRootRegion()
-   */
-  public static final String IS_ROOT = "IS_ROOT";
-  private static final Bytes IS_ROOT_KEY =
-      new Bytes(Bytes.toBytes(IS_ROOT));
-
-  /**
-   * <em>INTERNAL</em> Used by rest interface to access this metadata
-   * attribute which denotes if it is a catalog table, either
-   * <code> hbase:meta </code> or <code> -ROOT- </code>
-   *
-   * @see #isMetaRegion()
-   */
-  public static final String IS_META = "IS_META";
-  private static final Bytes IS_META_KEY =
-      new Bytes(Bytes.toBytes(IS_META));
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which denotes if the deferred log flush option is enabled.
-   * @deprecated Use {@link #DURABILITY} instead.
-   */
-  @Deprecated
-  public static final String DEFERRED_LOG_FLUSH = "DEFERRED_LOG_FLUSH";
-  @Deprecated
-  private static final Bytes DEFERRED_LOG_FLUSH_KEY =
-      new Bytes(Bytes.toBytes(DEFERRED_LOG_FLUSH));
-
-  /**
-   * <em>INTERNAL</em> {@link Durability} setting for the table.
-   */
-  public static final String DURABILITY = "DURABILITY";
-  private static final Bytes DURABILITY_KEY =
-      new Bytes(Bytes.toBytes("DURABILITY"));
-
-  /**
-   * <em>INTERNAL</em> number of region replicas for the table.
-   */
-  public static final String REGION_REPLICATION = "REGION_REPLICATION";
-  private static final Bytes REGION_REPLICATION_KEY =
-      new Bytes(Bytes.toBytes(REGION_REPLICATION));
-
-  /**
-   * <em>INTERNAL</em> flag to indicate whether or not the memstore should be replicated
-   * for read-replicas (CONSISTENCY =&gt; TIMELINE).
-   */
-  public static final String REGION_MEMSTORE_REPLICATION = "REGION_MEMSTORE_REPLICATION";
-  private static final Bytes REGION_MEMSTORE_REPLICATION_KEY =
-      new Bytes(Bytes.toBytes(REGION_MEMSTORE_REPLICATION));
-
-  /**
-   * <em>INTERNAL</em> Used by shell/rest interface to access this metadata
-   * attribute which denotes if the table should be treated by region normalizer.
-   *
-   * @see #isNormalizationEnabled()
-   */
-  public static final String NORMALIZATION_ENABLED = "NORMALIZATION_ENABLED";
-  private static final Bytes NORMALIZATION_ENABLED_KEY =
-    new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
-
-  /** Default durability for HTD is USE_DEFAULT, which defaults to HBase-global default value */
-  private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
-
-  public static final String PRIORITY = "PRIORITY";
-  private static final Bytes PRIORITY_KEY =
-    new Bytes(Bytes.toBytes(PRIORITY));
-
-  /** Relative priority of the table used for rpc scheduling */
-  private static final int DEFAULT_PRIORITY = HConstants.NORMAL_QOS;
-
-  /*
-   *  The below are ugly but better than creating them each time till we
-   *  replace booleans being saved as Strings with plain booleans.  Need a
-   *  migration script to do this.  TODO.
-   */
-  private static final Bytes FALSE =
-      new Bytes(Bytes.toBytes(Boolean.FALSE.toString()));
-
-  private static final Bytes TRUE =
-      new Bytes(Bytes.toBytes(Boolean.TRUE.toString()));
-
-  private static final boolean DEFAULT_DEFERRED_LOG_FLUSH = false;
-
-  /**
-   * Constant that denotes whether the table is READONLY by default and is false
-   */
-  public static final boolean DEFAULT_READONLY = false;
-
-  /**
-   * Constant that denotes whether the table is compaction enabled by default
-   */
-  public static final boolean DEFAULT_COMPACTION_ENABLED = true;
-
-  /**
-   * Constant that denotes whether the table is normalized by default.
-   */
-  public static final boolean DEFAULT_NORMALIZATION_ENABLED = false;
-
-  /**
-   * Constant that denotes the maximum default size of the memstore after which
-   * the contents are flushed to the store files
-   */
-  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = 1024*1024*128L;
-
-  public static final int DEFAULT_REGION_REPLICATION = 1;
-
-  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true;
-
-  private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
-  private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
-
-  static {
-    DEFAULT_VALUES.put(MAX_FILESIZE,
-        String.valueOf(HConstants.DEFAULT_MAX_FILE_SIZE));
-    DEFAULT_VALUES.put(READONLY, String.valueOf(DEFAULT_READONLY));
-    DEFAULT_VALUES.put(MEMSTORE_FLUSHSIZE,
-        String.valueOf(DEFAULT_MEMSTORE_FLUSH_SIZE));
-    DEFAULT_VALUES.put(DEFERRED_LOG_FLUSH,
-        String.valueOf(DEFAULT_DEFERRED_LOG_FLUSH));
-    DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); //use the enum name
-    DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION));
-    DEFAULT_VALUES.put(NORMALIZATION_ENABLED, String.valueOf(DEFAULT_NORMALIZATION_ENABLED));
-    DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY));
-    for (String s : DEFAULT_VALUES.keySet()) {
-      RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(s)));
-    }
-    RESERVED_KEYWORDS.add(IS_ROOT_KEY);
-    RESERVED_KEYWORDS.add(IS_META_KEY);
-  }
-
-  /**
-   * Cache of whether this is a meta table or not.
-   */
-  private volatile Boolean meta = null;
-  /**
-   * Cache of whether this is root table or not.
-   */
-  private volatile Boolean root = null;
-
-  /**
-   * Durability setting for the table
-   */
-  private Durability durability = null;
-
-  /**
-   * Maps column family name to the respective HColumnDescriptors
-   */
-  private final Map<byte [], HColumnDescriptor> families =
-    new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
-
-  /**
-   * <em> INTERNAL </em> Private constructor used internally creating table descriptors for
-   * catalog tables, <code>hbase:meta</code> and <code>-ROOT-</code>.
-   */
-  @InterfaceAudience.Private
-  protected HTableDescriptor(final TableName name, HColumnDescriptor[] families) {
-    setName(name);
-    for(HColumnDescriptor descriptor : families) {
-      this.families.put(descriptor.getName(), descriptor);
-    }
-  }
-
-  /**
-   * <em> INTERNAL </em>Private constructor used internally creating table descriptors for
-   * catalog tables, <code>hbase:meta</code> and <code>-ROOT-</code>.
-   */
-  protected HTableDescriptor(final TableName name, HColumnDescriptor[] families,
-      Map<Bytes, Bytes> values) {
-    setName(name);
-    for(HColumnDescriptor descriptor : families) {
-      this.families.put(descriptor.getName(), descriptor);
-    }
-    for (Map.Entry<Bytes, Bytes> entry :
-        values.entrySet()) {
-      setValue(entry.getKey(), entry.getValue());
-    }
-  }
-
-  /**
-   * Default constructor which constructs an empty object.
-   * For deserializing an HTableDescriptor instance only.
-   * @deprecated As of release 0.96 (<a href="https://issues.apache.org/jira/browse/HBASE-5453">HBASE-5453</a>).
-   *             This was made protected in 2.0.0 and will be removed in HBase 3.0.0.
-   *             Used by Writables and Writables are going away.
-   */
-  @Deprecated
-  protected HTableDescriptor() {
-    super();
-  }
+public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescriptor> {
+  public static final String SPLIT_POLICY = TableDescriptorBuilder.SPLIT_POLICY;
+  public static final String MAX_FILESIZE = TableDescriptorBuilder.MAX_FILESIZE;
+  public static final String OWNER = TableDescriptorBuilder.OWNER;
+  public static final Bytes OWNER_KEY = TableDescriptorBuilder.OWNER_KEY;
+  public static final String READONLY = TableDescriptorBuilder.READONLY;
+  public static final String COMPACTION_ENABLED = TableDescriptorBuilder.COMPACTION_ENABLED;
+  public static final String MEMSTORE_FLUSHSIZE = TableDescriptorBuilder.MEMSTORE_FLUSHSIZE;
+  public static final String FLUSH_POLICY = TableDescriptorBuilder.FLUSH_POLICY;
+  public static final String IS_ROOT = TableDescriptorBuilder.IS_ROOT;
+  public static final String IS_META = TableDescriptorBuilder.IS_META;
+  public static final String DURABILITY = TableDescriptorBuilder.DURABILITY;
+  public static final String REGION_REPLICATION = TableDescriptorBuilder.REGION_REPLICATION;
+  public static final String REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION;
+  public static final String NORMALIZATION_ENABLED = TableDescriptorBuilder.NORMALIZATION_ENABLED;
+  public static final String PRIORITY = TableDescriptorBuilder.PRIORITY;
+  public static final boolean DEFAULT_READONLY = TableDescriptorBuilder.DEFAULT_READONLY;
+  public static final boolean DEFAULT_COMPACTION_ENABLED = TableDescriptorBuilder.DEFAULT_COMPACTION_ENABLED;
+  public static final boolean DEFAULT_NORMALIZATION_ENABLED = TableDescriptorBuilder.DEFAULT_NORMALIZATION_ENABLED;
+  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE;
+  public static final int DEFAULT_REGION_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
+  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_MEMSTORE_REPLICATION;
+  private final ModifyableTableDescriptor delegatee;
 
   /**
    * Construct a table descriptor specifying a TableName object
@@ -321,39 +75,18 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 HBASE: (HBASE-174) Un-openable tablename bug</a>
    */
   public HTableDescriptor(final TableName name) {
-    super();
-    setName(name);
-  }
-
-  /**
-   * Construct a table descriptor specifying a byte array table name
-   * @param name Table name.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 (HBASE-174) HBASE: Un-openable tablename bug</a>
-   */
-  @Deprecated
-  public HTableDescriptor(final byte[] name) {
-    this(TableName.valueOf(name));
-  }
-
-  /**
-   * Construct a table descriptor specifying a String table name
-   * @param name Table name.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 (HBASE-174) HBASE: Un-openable tablename bug</a>
-   */
-  @Deprecated
-  public HTableDescriptor(final String name) {
-    this(TableName.valueOf(name));
+    this(name, Collections.EMPTY_LIST, Collections.EMPTY_MAP, Collections.EMPTY_MAP);
   }
 
   /**
    * Construct a table descriptor by cloning the descriptor passed as a parameter.
    * <p>
    * Makes a deep copy of the supplied descriptor.
-   * Can make a modifiable descriptor from an UnmodifyableHTableDescriptor.
+   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
    * @param desc The descriptor.
    */
   public HTableDescriptor(final HTableDescriptor desc) {
-    this(desc.name, desc);
+    this(desc.getTableName(), desc.getFamilies(), desc.getValues(), desc.getConfiguration());
   }
 
   /**
@@ -361,36 +94,25 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * but using a different table name.
    * <p>
    * Makes a deep copy of the supplied descriptor.
-   * Can make a modifiable descriptor from an UnmodifyableHTableDescriptor.
+   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
    * @param name Table name.
    * @param desc The descriptor.
    */
   public HTableDescriptor(final TableName name, final HTableDescriptor desc) {
-    super();
-    setName(name);
-    setMetaFlags(this.name);
-    for (HColumnDescriptor c: desc.families.values()) {
-      this.families.put(c.getName(), new HColumnDescriptor(c));
-    }
-    for (Map.Entry<Bytes, Bytes> e :
-        desc.values.entrySet()) {
-      setValue(e.getKey(), e.getValue());
-    }
-    for (Map.Entry<String, String> e : desc.configuration.entrySet()) {
-      this.configuration.put(e.getKey(), e.getValue());
-    }
+    this(name, desc.getFamilies(), desc.getValues(), desc.getConfiguration());
   }
 
-  /*
-   * Set meta flags on this table.
-   * IS_ROOT_KEY is set if its a -ROOT- table
-   * IS_META_KEY is set either if its a -ROOT- or a hbase:meta table
-   * Called by constructors.
-   * @param name
-   */
-  private void setMetaFlags(final TableName name) {
-    setMetaRegion(isRootRegion() ||
-        name.equals(TableName.META_TABLE_NAME));
+  public HTableDescriptor(final TableDescriptor desc) {
+    this(desc.getTableName(), desc.getFamilies(), desc.getValues(), desc.getConfiguration());
+  }
+
+  private HTableDescriptor(final TableName name, final Collection<HColumnDescriptor> families,
+      Map<Bytes, Bytes> values, Map<String, String> configuration) {
+    this(new ModifyableTableDescriptor(name, families, values, configuration));
+  }
+
+  protected HTableDescriptor(ModifyableTableDescriptor delegatee) {
+    this.delegatee = delegatee;
   }
 
   /**
@@ -398,23 +120,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return true if this is a <code> -ROOT- </code> region
    */
+  @Override
   public boolean isRootRegion() {
-    if (this.root == null) {
-      this.root = isSomething(IS_ROOT_KEY, false)? Boolean.TRUE: Boolean.FALSE;
-    }
-    return this.root.booleanValue();
-  }
-
-  /**
-   * <em> INTERNAL </em> Used to denote if the current table represents
-   * <code> -ROOT- </code> region. This is used internally by the
-   * HTableDescriptor constructors
-   *
-   * @param isRoot true if this is the <code> -ROOT- </code> region
-   */
-  protected void setRootRegion(boolean isRoot) {
-    // TODO: Make the value a boolean rather than String of boolean.
-    setValue(IS_ROOT_KEY, isRoot? TRUE: FALSE);
+    return delegatee.isRootRegion();
   }
 
   /**
@@ -424,37 +132,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return true if this table is <code> hbase:meta </code>
    * region
    */
+  @Override
   public boolean isMetaRegion() {
-    if (this.meta == null) {
-      this.meta = calculateIsMetaRegion();
-    }
-    return this.meta.booleanValue();
-  }
-
-  private synchronized Boolean calculateIsMetaRegion() {
-    byte [] value = getValue(IS_META_KEY);
-    return (value != null)? Boolean.valueOf(Bytes.toString(value)): Boolean.FALSE;
-  }
-
-  private boolean isSomething(final Bytes key,
-      final boolean valueIfNull) {
-    byte [] value = getValue(key);
-    if (value != null) {
-      return Boolean.valueOf(Bytes.toString(value));
-    }
-    return valueIfNull;
-  }
-
-  /**
-   * <em> INTERNAL </em> Used to denote if the current table represents
-   * <code> -ROOT- </code> or <code> hbase:meta </code> region. This is used
-   * internally by the HTableDescriptor constructors
-   *
-   * @param isMeta true if its either <code> -ROOT- </code> or
-   * <code> hbase:meta </code> region
-   */
-  protected void setMetaRegion(boolean isMeta) {
-    setValue(IS_META_KEY, isMeta? TRUE: FALSE);
+    return delegatee.isMetaRegion();
   }
 
   /**
@@ -462,8 +142,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return true if table is <code> hbase:meta </code> region.
    */
+  @Override
   public boolean isMetaTable() {
-    return isMetaRegion() && !isRootRegion();
+    return delegatee.isMetaTable();
   }
 
   /**
@@ -471,17 +152,10 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @param key The key.
    * @return The value.
-   * @see #values
    */
+  @Override
   public byte[] getValue(byte[] key) {
-    return getValue(new Bytes(key));
-  }
-
-  private byte[] getValue(final Bytes key) {
-    Bytes ibw = values.get(key);
-    if (ibw == null)
-      return null;
-    return ibw.get();
+    return delegatee.getValue(key);
   }
 
   /**
@@ -489,110 +163,80 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @param key The key.
    * @return The value.
-   * @see #values
    */
   public String getValue(String key) {
-    byte[] value = getValue(Bytes.toBytes(key));
-    if (value == null)
-      return null;
-    return Bytes.toString(value);
+    return delegatee.getValue(key);
   }
 
   /**
-   * Getter for fetching an unmodifiable {@link #values} map.
-   *
-   * @return unmodifiable map {@link #values}.
-   * @see #values
+   * @return Getter for fetching an unmodifiable map.
    */
+  @Override
   public Map<Bytes, Bytes> getValues() {
-    // shallow pointer copy
-    return Collections.unmodifiableMap(values);
+    return delegatee.getValues();
   }
 
   /**
-   * Setter for storing metadata as a (key, value) pair in {@link #values} map
+   * Setter for storing metadata as a (key, value) pair in map
    *
    * @param key The key.
-   * @param value The value.
-   * @see #values
+   * @param value The value. If null, removes the setting.
    */
   public HTableDescriptor setValue(byte[] key, byte[] value) {
-    setValue(new Bytes(key), new Bytes(value));
-    return this;
-  }
-
-  /*
-   * @param key The key.
-   * @param value The value.
-   */
-  private HTableDescriptor setValue(final Bytes key,
-      final String value) {
-    setValue(key, new Bytes(Bytes.toBytes(value)));
+    delegatee.setValue(key, value);
     return this;
   }
 
   /*
-   * Setter for storing metadata as a (key, value) pair in {@link #values} map
+   * Setter for storing metadata as a (key, value) pair in map
    *
    * @param key The key.
-   * @param value The value.
+   * @param value The value. If null, removes the setting.
    */
   public HTableDescriptor setValue(final Bytes key, final Bytes value) {
-    if (key.compareTo(DEFERRED_LOG_FLUSH_KEY) == 0) {
-      boolean isDeferredFlush = Boolean.valueOf(Bytes.toString(value.get()));
-      LOG.warn("HTableDescriptor property:" + DEFERRED_LOG_FLUSH + " is deprecated, " +
-          "use " + DURABILITY + " instead");
-      setDurability(isDeferredFlush ? Durability.ASYNC_WAL : DEFAULT_DURABLITY);
-      return this;
-    }
-    values.put(key, value);
+    delegatee.setValue(key, value);
     return this;
   }
 
   /**
-   * Setter for storing metadata as a (key, value) pair in {@link #values} map
+   * Setter for storing metadata as a (key, value) pair in map
    *
    * @param key The key.
-   * @param value The value.
-   * @see #values
+   * @param value The value. If null, removes the setting.
    */
   public HTableDescriptor setValue(String key, String value) {
-    if (value == null) {
-      remove(key);
-    } else {
-      setValue(Bytes.toBytes(key), Bytes.toBytes(value));
-    }
+    delegatee.setValue(key, value);
     return this;
   }
 
   /**
-   * Remove metadata represented by the key from the {@link #values} map
+   * Remove metadata represented by the key from the map
    *
    * @param key Key whose key and value we're to remove from HTableDescriptor
    * parameters.
    */
   public void remove(final String key) {
-    remove(new Bytes(Bytes.toBytes(key)));
+    delegatee.remove(key);
   }
 
   /**
-   * Remove metadata represented by the key from the {@link #values} map
+   * Remove metadata represented by the key from the map
    *
    * @param key Key whose key and value we're to remove from HTableDescriptor
    * parameters.
    */
   public void remove(Bytes key) {
-    values.remove(key);
+    delegatee.remove(key);
   }
 
   /**
-   * Remove metadata represented by the key from the {@link #values} map
+   * Remove metadata represented by the key from the map
    *
    * @param key Key whose key and value we're to remove from HTableDescriptor
    * parameters.
    */
   public void remove(final byte [] key) {
-    remove(new Bytes(key));
+    delegatee.remove(key);
   }
 
   /**
@@ -601,8 +245,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return true if all columns in the table should be read only
    */
+  @Override
   public boolean isReadOnly() {
-    return isSomething(READONLY_KEY, DEFAULT_READONLY);
+    return delegatee.isReadOnly();
   }
 
   /**
@@ -614,7 +259,8 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * only.
    */
   public HTableDescriptor setReadOnly(final boolean readOnly) {
-    return setValue(READONLY_KEY, readOnly? TRUE: FALSE);
+    delegatee.setReadOnly(readOnly);
+    return this;
   }
 
   /**
@@ -623,8 +269,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return true if table compaction enabled
    */
+  @Override
   public boolean isCompactionEnabled() {
-    return isSomething(COMPACTION_ENABLED_KEY, DEFAULT_COMPACTION_ENABLED);
+    return delegatee.isCompactionEnabled();
   }
 
   /**
@@ -633,7 +280,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param isEnable True if enable compaction.
    */
   public HTableDescriptor setCompactionEnabled(final boolean isEnable) {
-    setValue(COMPACTION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    delegatee.setCompactionEnabled(isEnable);
     return this;
   }
 
@@ -643,8 +290,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return true if region normalization is enabled for this table
    */
+  @Override
   public boolean isNormalizationEnabled() {
-    return isSomething(NORMALIZATION_ENABLED_KEY, DEFAULT_NORMALIZATION_ENABLED);
+    return delegatee.isNormalizationEnabled();
   }
 
   /**
@@ -653,7 +301,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param isEnable True if enable normalization.
    */
   public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
-    setValue(NORMALIZATION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    delegatee.setNormalizationEnabled(isEnable);
     return this;
   }
 
@@ -662,8 +310,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param durability enum value
    */
   public HTableDescriptor setDurability(Durability durability) {
-    this.durability = durability;
-    setValue(DURABILITY_KEY, durability.name());
+    delegatee.setDurability(durability);
     return this;
   }
 
@@ -671,22 +318,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * Returns the durability setting for the table.
    * @return durability setting for the table.
    */
+  @Override
   public Durability getDurability() {
-    if (this.durability == null) {
-      byte[] durabilityValue = getValue(DURABILITY_KEY);
-      if (durabilityValue == null) {
-        this.durability = DEFAULT_DURABLITY;
-      } else {
-        try {
-          this.durability = Durability.valueOf(Bytes.toString(durabilityValue));
-        } catch (IllegalArgumentException ex) {
-          LOG.warn("Received " + ex + " because Durability value for HTableDescriptor"
-            + " is not known. Durability:" + Bytes.toString(durabilityValue));
-          this.durability = DEFAULT_DURABLITY;
-        }
-      }
-    }
-    return this.durability;
+    return delegatee.getDurability();
   }
 
   /**
@@ -694,19 +328,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return TableName
    */
+  @Override
   public TableName getTableName() {
-    return name;
-  }
-
-  /**
-   * Get the name of the table as a byte array.
-   *
-   * @return name of table
-   * @deprecated Use {@link #getTableName()} instead
-   */
-  @Deprecated
-  public byte[] getName() {
-    return name.getName();
+    return delegatee.getTableName();
   }
 
   /**
@@ -715,7 +339,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return name of table as a String
    */
   public String getNameAsString() {
-    return name.getNameAsString();
+    return delegatee.getTableName().getNameAsString();
   }
 
   /**
@@ -725,7 +349,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param clazz the class name
    */
   public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
-    setValue(SPLIT_POLICY, clazz);
+    delegatee.setRegionSplitPolicyClassName(clazz);
     return this;
   }
 
@@ -737,26 +361,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return the class name of the region split policy for this table.
    * If this returns null, the default split policy is used.
    */
+  @Override
    public String getRegionSplitPolicyClassName() {
-    return getValue(SPLIT_POLICY);
-  }
-
-  /**
-   * Set the name of the table.
-   *
-   * @param name name of table
-   */
-  @Deprecated
-  public HTableDescriptor setName(byte[] name) {
-    setName(TableName.valueOf(name));
-    return this;
-  }
-
-  @Deprecated
-  public HTableDescriptor setName(TableName name) {
-    this.name = name;
-    setMetaFlags(this.name);
-    return this;
+    return delegatee.getRegionSplitPolicyClassName();
   }
 
   /**
@@ -768,12 +375,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @see #setMaxFileSize(long)
    */
+   @Override
   public long getMaxFileSize() {
-    byte [] value = getValue(MAX_FILESIZE_KEY);
-    if (value != null) {
-      return Long.parseLong(Bytes.toString(value));
-    }
-    return -1;
+    return delegatee.getMaxFileSize();
   }
 
   /**
@@ -792,7 +396,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * before a split is triggered.
    */
   public HTableDescriptor setMaxFileSize(long maxFileSize) {
-    setValue(MAX_FILESIZE_KEY, Long.toString(maxFileSize));
+    delegatee.setMaxFileSize(maxFileSize);
     return this;
   }
 
@@ -803,12 +407,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @see #setMemStoreFlushSize(long)
    */
+  @Override
   public long getMemStoreFlushSize() {
-    byte [] value = getValue(MEMSTORE_FLUSHSIZE_KEY);
-    if (value != null) {
-      return Long.parseLong(Bytes.toString(value));
-    }
-    return -1;
+    return delegatee.getMemStoreFlushSize();
   }
 
   /**
@@ -818,7 +419,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param memstoreFlushSize memory cache flush size for each hregion
    */
   public HTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
-    setValue(MEMSTORE_FLUSHSIZE_KEY, Long.toString(memstoreFlushSize));
+    delegatee.setMemStoreFlushSize(memstoreFlushSize);
     return this;
   }
 
@@ -829,7 +430,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param clazz the class name
    */
   public HTableDescriptor setFlushPolicyClassName(String clazz) {
-    setValue(FLUSH_POLICY, clazz);
+    delegatee.setFlushPolicyClassName(clazz);
     return this;
   }
 
@@ -840,8 +441,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return the class name of the flush policy for this table. If this returns null, the default
    *         flush policy is used.
    */
+  @Override
   public String getFlushPolicyClassName() {
-    return getValue(FLUSH_POLICY);
+    return delegatee.getFlushPolicyClassName();
   }
 
   /**
@@ -850,14 +452,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param family HColumnDescriptor of family to add.
    */
   public HTableDescriptor addFamily(final HColumnDescriptor family) {
-    if (family.getName() == null || family.getName().length <= 0) {
-      throw new IllegalArgumentException("Family name cannot be null or empty");
-    }
-    if (hasFamily(family.getName())) {
-      throw new IllegalArgumentException("Family '" +
-        family.getNameAsString() + "' already exists so cannot be added");
-    }
-    this.families.put(family.getName(), family);
+    delegatee.addFamily(family);
     return this;
   }
 
@@ -867,14 +462,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return this (for chained invocation)
    */
   public HTableDescriptor modifyFamily(final HColumnDescriptor family) {
-    if (family.getName() == null || family.getName().length <= 0) {
-      throw new IllegalArgumentException("Family name cannot be null or empty");
-    }
-    if (!hasFamily(family.getName())) {
-      throw new IllegalArgumentException("Column family '" + family.getNameAsString()
-        + "' does not exist");
-    }
-    this.families.put(family.getName(), family);
+    delegatee.modifyFamily(family);
     return this;
   }
 
@@ -883,8 +471,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param familyName Family name or column name.
    * @return true if the table contains the specified family name
    */
+  @Override
   public boolean hasFamily(final byte [] familyName) {
-    return families.containsKey(familyName);
+    return delegatee.hasFamily(familyName);
   }
 
   /**
@@ -894,13 +483,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   @Override
   public String toString() {
-    StringBuilder s = new StringBuilder();
-    s.append('\'').append(Bytes.toString(name.getName())).append('\'');
-    s.append(getValues(true));
-    for (HColumnDescriptor f : families.values()) {
-      s.append(", ").append(f);
-    }
-    return s.toString();
+    return delegatee.toString();
   }
 
   /**
@@ -908,109 +491,14 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * descriptors (with only the non-default column family attributes)
    */
   public String toStringCustomizedValues() {
-    StringBuilder s = new StringBuilder();
-    s.append('\'').append(Bytes.toString(name.getName())).append('\'');
-    s.append(getValues(false));
-    for(HColumnDescriptor hcd : families.values()) {
-      s.append(", ").append(hcd.toStringCustomizedValues());
-    }
-    return s.toString();
+    return delegatee.toStringCustomizedValues();
   }
 
   /**
    * @return map of all table attributes formatted into string.
    */
   public String toStringTableAttributes() {
-   return getValues(true).toString();
-  }
-
-  private StringBuilder getValues(boolean printDefaults) {
-    StringBuilder s = new StringBuilder();
-
-    // step 1: set partitioning and pruning
-    Set<Bytes> reservedKeys = new TreeSet<>();
-    Set<Bytes> userKeys = new TreeSet<>();
-    for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
-      if (entry.getKey() == null || entry.getKey().get() == null) continue;
-      String key = Bytes.toString(entry.getKey().get());
-      // in this section, print out reserved keywords + coprocessor info
-      if (!RESERVED_KEYWORDS.contains(entry.getKey()) && !key.startsWith("coprocessor$")) {
-        userKeys.add(entry.getKey());
-        continue;
-      }
-      // only print out IS_ROOT/IS_META if true
-      String value = Bytes.toString(entry.getValue().get());
-      if (key.equalsIgnoreCase(IS_ROOT) || key.equalsIgnoreCase(IS_META)) {
-        if (Boolean.valueOf(value) == false) continue;
-      }
-      // see if a reserved key is a default value. may not want to print it out
-      if (printDefaults
-          || !DEFAULT_VALUES.containsKey(key)
-          || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
-        reservedKeys.add(entry.getKey());
-      }
-    }
-
-    // early exit optimization
-    boolean hasAttributes = !reservedKeys.isEmpty() || !userKeys.isEmpty();
-    if (!hasAttributes && configuration.isEmpty()) return s;
-
-    s.append(", {");
-    // step 2: printing attributes
-    if (hasAttributes) {
-      s.append("TABLE_ATTRIBUTES => {");
-
-      // print all reserved keys first
-      boolean printCommaForAttr = false;
-      for (Bytes k : reservedKeys) {
-        String key = Bytes.toString(k.get());
-        String value = Bytes.toStringBinary(values.get(k).get());
-        if (printCommaForAttr) s.append(", ");
-        printCommaForAttr = true;
-        s.append(key);
-        s.append(" => ");
-        s.append('\'').append(value).append('\'');
-      }
-
-      if (!userKeys.isEmpty()) {
-        // print all non-reserved, advanced config keys as a separate subset
-        if (printCommaForAttr) s.append(", ");
-        printCommaForAttr = true;
-        s.append(HConstants.METADATA).append(" => ");
-        s.append("{");
-        boolean printCommaForCfg = false;
-        for (Bytes k : userKeys) {
-          String key = Bytes.toString(k.get());
-          String value = Bytes.toStringBinary(values.get(k).get());
-          if (printCommaForCfg) s.append(", ");
-          printCommaForCfg = true;
-          s.append('\'').append(key).append('\'');
-          s.append(" => ");
-          s.append('\'').append(value).append('\'');
-        }
-        s.append("}");
-      }
-    }
-
-    // step 3: printing all configuration:
-    if (!configuration.isEmpty()) {
-      if (hasAttributes) {
-        s.append(", ");
-      }
-      s.append(HConstants.CONFIGURATION).append(" => ");
-      s.append('{');
-      boolean printCommaForConfig = false;
-      for (Map.Entry<String, String> e : configuration.entrySet()) {
-        if (printCommaForConfig) s.append(", ");
-        printCommaForConfig = true;
-        s.append('\'').append(e.getKey()).append('\'');
-        s.append(" => ");
-        s.append('\'').append(e.getValue()).append('\'');
-      }
-      s.append("}");
-    }
-    s.append("}"); // end METHOD
-    return s;
+   return delegatee.toStringTableAttributes();
   }
 
   /**
@@ -1041,15 +529,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   @Override
   public int hashCode() {
-    int result = this.name.hashCode();
-    if (this.families.size() > 0) {
-      for (HColumnDescriptor e: this.families.values()) {
-        result ^= e.hashCode();
-      }
-    }
-    result ^= values.hashCode();
-    result ^= configuration.hashCode();
-    return result;
+    return delegatee.hashCode();
   }
 
   // Comparable
@@ -1063,39 +543,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   @Override
   public int compareTo(final HTableDescriptor other) {
-    int result = this.name.compareTo(other.name);
-    if (result == 0) {
-      result = families.size() - other.families.size();
-    }
-    if (result == 0 && families.size() != other.families.size()) {
-      result = Integer.valueOf(families.size()).compareTo(
-          Integer.valueOf(other.families.size()));
-    }
-    if (result == 0) {
-      for (Iterator<HColumnDescriptor> it = families.values().iterator(),
-          it2 = other.families.values().iterator(); it.hasNext(); ) {
-        result = it.next().compareTo(it2.next());
-        if (result != 0) {
-          break;
-        }
-      }
-    }
-    if (result == 0) {
-      // punt on comparison for ordering, just calculate difference
-      result = this.values.hashCode() - other.values.hashCode();
-      if (result < 0)
-        result = -1;
-      else if (result > 0)
-        result = 1;
-    }
-    if (result == 0) {
-      result = this.configuration.hashCode() - other.configuration.hashCode();
-      if (result < 0)
-        result = -1;
-      else if (result > 0)
-        result = 1;
-    }
-    return result;
+    return delegatee.compareTo(other.delegatee);
   }
 
   /**
@@ -1105,35 +553,25 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return Immutable collection of {@link HColumnDescriptor} of all the
    * column families.
    */
+  @Override
   public Collection<HColumnDescriptor> getFamilies() {
-    return Collections.unmodifiableCollection(this.families.values());
+    return delegatee.getFamilies();
   }
 
   /**
    * Return true if there are at least one cf whose replication scope is serial.
    */
+  @Override
   public boolean hasSerialReplicationScope() {
-    for (HColumnDescriptor column: getFamilies()){
-      if (column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL){
-        return true;
-      }
-    }
-    return false;
+    return delegatee.hasSerialReplicationScope();
   }
 
   /**
    * Returns the configured replicas per region
    */
+  @Override
   public int getRegionReplication() {
-    return getIntValue(REGION_REPLICATION_KEY, DEFAULT_REGION_REPLICATION);
-  }
-
-  private int getIntValue(Bytes key, int defaultVal) {
-    byte[] val = getValue(key);
-    if (val == null || val.length == 0) {
-      return defaultVal;
-    }
-    return Integer.parseInt(Bytes.toString(val));
+    return delegatee.getRegionReplication();
   }
 
   /**
@@ -1141,16 +579,16 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param regionReplication the replication factor per region
    */
   public HTableDescriptor setRegionReplication(int regionReplication) {
-    setValue(REGION_REPLICATION_KEY,
-        new Bytes(Bytes.toBytes(Integer.toString(regionReplication))));
+    delegatee.setRegionReplication(regionReplication);
     return this;
   }
 
   /**
    * @return true if the read-replicas memstore replication is enabled.
    */
+  @Override
   public boolean hasRegionMemstoreReplication() {
-    return isSomething(REGION_MEMSTORE_REPLICATION_KEY, DEFAULT_REGION_MEMSTORE_REPLICATION);
+    return delegatee.hasRegionMemstoreReplication();
   }
 
   /**
@@ -1163,21 +601,18 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *                                  data only when the primary flushes the memstore.
    */
   public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
-    setValue(REGION_MEMSTORE_REPLICATION_KEY, memstoreReplication ? TRUE : FALSE);
-    // If the memstore replication is setup, we do not have to wait for observing a flush event
-    // from primary before starting to serve reads, because gaps from replication is not applicable
-    setConfiguration(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY,
-      Boolean.toString(memstoreReplication));
+    delegatee.setRegionMemstoreReplication(memstoreReplication);
     return this;
   }
 
   public HTableDescriptor setPriority(int priority) {
-    setValue(PRIORITY_KEY, Integer.toString(priority));
+    delegatee.setPriority(priority);
     return this;
   }
 
+  @Override
   public int getPriority() {
-    return getIntValue(PRIORITY_KEY, DEFAULT_PRIORITY);
+    return delegatee.getPriority();
   }
 
   /**
@@ -1188,8 +623,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return Immutable sorted set of the keys of the families.
    */
+  @Override
   public Set<byte[]> getFamiliesKeys() {
-    return Collections.unmodifiableSet(this.families.keySet());
+    return delegatee.getFamiliesKeys();
   }
 
   /**
@@ -1197,8 +633,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return Count of column families of the table
    */
+  @Override
   public int getColumnFamilyCount() {
-    return families.size();
+    return delegatee.getColumnFamilyCount();
   }
 
   /**
@@ -1209,9 +646,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @see #getFamilies()
    */
+  @Override
   public HColumnDescriptor[] getColumnFamilies() {
-    Collection<HColumnDescriptor> hColumnDescriptors = getFamilies();
-    return hColumnDescriptors.toArray(new HColumnDescriptor[hColumnDescriptors.size()]);
+    return delegatee.getColumnFamilies();
   }
 
 
@@ -1223,8 +660,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return Column descriptor for the passed family name or the family on
    * passed in column.
    */
+  @Override
   public HColumnDescriptor getFamily(final byte [] column) {
-    return this.families.get(column);
+    return delegatee.getFamily(column);
   }
 
 
@@ -1237,7 +675,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * passed in column.
    */
   public HColumnDescriptor removeFamily(final byte [] column) {
-    return this.families.remove(column);
+    return delegatee.removeFamily(column);
   }
 
   /**
@@ -1251,7 +689,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @throws IOException
    */
   public HTableDescriptor addCoprocessor(String className) throws IOException {
-    addCoprocessor(className, null, Coprocessor.PRIORITY_USER, null);
+    delegatee.addCoprocessor(className);
     return this;
   }
 
@@ -1272,32 +710,8 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
   public HTableDescriptor addCoprocessor(String className, Path jarFilePath,
                              int priority, final Map<String, String> kvs)
   throws IOException {
-    checkHasCoprocessor(className);
-
-    // Validate parameter kvs and then add key/values to kvString.
-    StringBuilder kvString = new StringBuilder();
-    if (kvs != null) {
-      for (Map.Entry<String, String> e: kvs.entrySet()) {
-        if (!e.getKey().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_KEY_PATTERN)) {
-          throw new IOException("Illegal parameter key = " + e.getKey());
-        }
-        if (!e.getValue().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN)) {
-          throw new IOException("Illegal parameter (" + e.getKey() +
-              ") value = " + e.getValue());
-        }
-        if (kvString.length() != 0) {
-          kvString.append(',');
-        }
-        kvString.append(e.getKey());
-        kvString.append('=');
-        kvString.append(e.getValue());
-      }
-    }
-
-    String value = ((jarFilePath == null)? "" : jarFilePath.toString()) +
-        "|" + className + "|" + Integer.toString(priority) + "|" +
-        kvString.toString();
-    return addCoprocessorToMap(value);
+    delegatee.addCoprocessor(className, jarFilePath, priority, kvs);
+    return this;
   }
 
   /**
@@ -1312,42 +726,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @throws IOException
    */
   public HTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
-    String className = getCoprocessorClassNameFromSpecStr(specStr);
-    if (className == null) {
-      throw new IllegalArgumentException("Format does not match " +
-        HConstants.CP_HTD_ATTR_VALUE_PATTERN + ": " + specStr);
-    }
-    checkHasCoprocessor(className);
-    return addCoprocessorToMap(specStr);
-  }
-
-  private void checkHasCoprocessor(final String className) throws IOException {
-    if (hasCoprocessor(className)) {
-      throw new IOException("Coprocessor " + className + " already exists.");
-    }
-  }
-
-  /**
-   * Add coprocessor to values Map
-   * @param specStr The Coprocessor specification all in in one String formatted so matches
-   * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
-   * @return Returns <code>this</code>
-   */
-  private HTableDescriptor addCoprocessorToMap(final String specStr) {
-    if (specStr == null) return this;
-    // generate a coprocessor key
-    int maxCoprocessorNumber = 0;
-    Matcher keyMatcher;
-    for (Map.Entry<Bytes, Bytes> e: this.values.entrySet()) {
-      keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      maxCoprocessorNumber = Math.max(Integer.parseInt(keyMatcher.group(1)), maxCoprocessorNumber);
-    }
-    maxCoprocessorNumber++;
-    String key = "coprocessor$" + Integer.toString(maxCoprocessorNumber);
-    this.values.put(new Bytes(Bytes.toBytes(key)), new Bytes(Bytes.toBytes(specStr)));
+    delegatee.addCoprocessorWithSpec(specStr);
     return this;
   }
 
@@ -1357,23 +736,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param classNameToMatch - Class name of the co-processor
    * @return true of the table has a co-processor className
    */
+  @Override
   public boolean hasCoprocessor(String classNameToMatch) {
-    Matcher keyMatcher;
-    for (Map.Entry<Bytes, Bytes> e :
-        this.values.entrySet()) {
-      keyMatcher =
-          HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(
-              Bytes.toString(e.getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
-      if (className == null) continue;
-      if (className.equals(classNameToMatch.trim())) {
-        return true;
-      }
-    }
-    return false;
+    return delegatee.hasCoprocessor(classNameToMatch);
   }
 
   /**
@@ -1381,29 +746,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return The list of co-processors classNames
    */
+  @Override
   public List<String> getCoprocessors() {
-    List<String> result = new ArrayList<>(this.values.entrySet().size());
-    Matcher keyMatcher;
-    for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
-      keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
-      if (className == null) continue;
-      result.add(className); // classname is the 2nd field
-    }
-    return result;
-  }
-
-  /**
-   * @param spec String formatted as per {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
-   * @return Class parsed from passed in <code>spec</code> or null if no match or classpath found
-   */
-  private static String getCoprocessorClassNameFromSpecStr(final String spec) {
-    Matcher matcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(spec);
-    // Classname is the 2nd field
-    return matcher != null && matcher.matches()? matcher.group(2).trim(): null;
+    return delegatee.getCoprocessors();
   }
 
   /**
@@ -1411,97 +756,34 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param className Class name of the co-processor
    */
   public void removeCoprocessor(String className) {
-    Bytes match = null;
-    Matcher keyMatcher;
-    Matcher valueMatcher;
-    for (Map.Entry<Bytes, Bytes> e : this.values
-        .entrySet()) {
-      keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e
-          .getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes
-          .toString(e.getValue().get()));
-      if (!valueMatcher.matches()) {
-        continue;
-      }
-      // get className and compare
-      String clazz = valueMatcher.group(2).trim(); // classname is the 2nd field
-      // remove the CP if it is present
-      if (clazz.equals(className.trim())) {
-        match = e.getKey();
-        break;
-      }
-    }
-    // if we found a match, remove it
-    if (match != null)
-      remove(match);
+    delegatee.removeCoprocessor(className);
   }
 
-  /**
-   * Returns the {@link Path} object representing the table directory under
-   * path rootdir
-   *
-   * Deprecated use FSUtils.getTableDir() instead.
-   *
-   * @param rootdir qualified path of HBase root directory
-   * @param tableName name of table
-   * @return {@link Path} for table
-   */
-  @Deprecated
-  public static Path getTableDir(Path rootdir, final byte [] tableName) {
-    //This is bad I had to mirror code from FSUTils.getTableDir since
-    //there is no module dependency between hbase-client and hbase-server
-    TableName name = TableName.valueOf(tableName);
-    return new Path(rootdir, new Path(HConstants.BASE_NAMESPACE_DIR,
-              new Path(name.getNamespaceAsString(), new Path(name.getQualifierAsString()))));
-  }
-
-  public final static String NAMESPACE_FAMILY_INFO = "info";
-  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO);
-  public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d");
+  public final static String NAMESPACE_FAMILY_INFO = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO;
+  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES;
+  public final static byte[] NAMESPACE_COL_DESC_BYTES = TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES;
 
   /** Table descriptor for namespace table */
-  public static final HTableDescriptor NAMESPACE_TABLEDESC = new HTableDescriptor(
-      TableName.NAMESPACE_TABLE_NAME,
-      new HColumnDescriptor[] {
-          new HColumnDescriptor(NAMESPACE_FAMILY_INFO)
-              // Ten is arbitrary number.  Keep versions to help debugging.
-              .setMaxVersions(10)
-              .setInMemory(true)
-              .setBlocksize(8 * 1024)
-              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-              // Enable cache of data blocks in L1 if more than one caching tier deployed:
-              // e.g. if using CombinedBlockCache (BucketCache).
-              .setCacheDataInL1(true)
-      });
+  public static final HTableDescriptor NAMESPACE_TABLEDESC
+    = new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
 
   @Deprecated
   public HTableDescriptor setOwner(User owner) {
-    return setOwnerString(owner != null ? owner.getShortName() : null);
+    delegatee.setOwner(owner);
+    return this;
   }
 
   // used by admin.rb:alter(table_name,*args) to update owner.
   @Deprecated
   public HTableDescriptor setOwnerString(String ownerString) {
-    if (ownerString != null) {
-      setValue(OWNER_KEY, ownerString);
-    } else {
-      remove(OWNER_KEY);
-    }
+    delegatee.setOwnerString(ownerString);
     return this;
   }
 
+  @Override
   @Deprecated
   public String getOwnerString() {
-    if (getValue(OWNER_KEY) != null) {
-      return Bytes.toString(getValue(OWNER_KEY));
-    }
-    // Note that every table should have an owner (i.e. should have OWNER_KEY set).
-    // hbase:meta and -ROOT- should return system user as owner, not null (see
-    // MasterFileSystem.java:bootstrap()).
-    return null;
+    return delegatee.getOwnerString();
   }
 
   /**
@@ -1509,7 +791,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @see #parseFrom(byte[])
    */
   public byte[] toByteArray() {
-    return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
+    return delegatee.toByteArray();
   }
 
   /**
@@ -1521,54 +803,39 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   public static HTableDescriptor parseFrom(final byte [] bytes)
   throws DeserializationException, IOException {
-    if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Expected PB encoded HTableDescriptor");
-    }
-    int pblen = ProtobufUtil.lengthOfPBMagic();
-    TableSchema.Builder builder = TableSchema.newBuilder();
-    TableSchema ts;
-    try {
-      ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
-      ts = builder.build();
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-    return ProtobufUtil.convertToHTableDesc(ts);
+    return new HTableDescriptor(ModifyableTableDescriptor.parseFrom(bytes));
   }
 
   /**
    * Getter for accessing the configuration value by key
    */
+  @Override
   public String getConfigurationValue(String key) {
-    return configuration.get(key);
+    return delegatee.getConfigurationValue(key);
   }
 
   /**
-   * Getter for fetching an unmodifiable {@link #configuration} map.
+   * Getter for fetching an unmodifiable map.
    */
+  @Override
   public Map<String, String> getConfiguration() {
-    // shallow pointer copy
-    return Collections.unmodifiableMap(configuration);
+    return delegatee.getConfiguration();
   }
 
   /**
-   * Setter for storing a configuration setting in {@link #configuration} map.
+   * Setter for storing a configuration setting in map.
    * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
    * @param value String value. If null, removes the setting.
    */
   public HTableDescriptor setConfiguration(String key, String value) {
-    if (value == null) {
-      removeConfiguration(key);
-    } else {
-      configuration.put(key, value);
-    }
+    delegatee.setConfiguration(key, value);
     return this;
   }
 
   /**
-   * Remove a config setting represented by the key from the {@link #configuration} map
+   * Remove a config setting represented by the key from the map
    */
   public void removeConfiguration(final String key) {
-    configuration.remove(key);
+    delegatee.removeConfiguration(key);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 3e767d2..decf81f 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
@@ -95,7 +95,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * List all the userspace tables.
    *
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   HTableDescriptor[] listTables() throws IOException;
@@ -104,7 +104,7 @@ public interface Admin extends Abortable, Closeable {
    * List all the userspace tables matching the given pattern.
    *
    * @param pattern The compiled regular expression to match against
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    * @see #listTables()
    */
@@ -125,7 +125,7 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param pattern The compiled regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    * @see #listTables()
    */
@@ -137,7 +137,7 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param regex The regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    * @see #listTables(java.util.regex.Pattern, boolean)
    */
@@ -192,7 +192,7 @@ public interface Admin extends Abortable, Closeable {
    * Method for getting the tableDescriptor
    *
    * @param tableName as a {@link TableName}
-   * @return the tableDescriptor
+   * @return the read-only tableDescriptor
    * @throws org.apache.hadoop.hbase.TableNotFoundException
    * @throws IOException if a remote or network exception occurs
    */
@@ -293,7 +293,8 @@ public interface Admin extends Abortable, Closeable {
    * #listTables(java.lang.String)} and {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
    *
    * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be deleted
+   * @return Table descriptors for tables that couldn't be deleted.
+   *         The return htds are read-only
    * @throws IOException
    * @see #deleteTables(java.util.regex.Pattern)
    * @see #deleteTable(org.apache.hadoop.hbase.TableName)
@@ -308,6 +309,7 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted
+   *         The return htds are read-only
    * @throws IOException
    */
   HTableDescriptor[] deleteTables(Pattern pattern) throws IOException;
@@ -373,6 +375,8 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param regex The regular expression to match table names against
    * @throws IOException
+   * @return Table descriptors for tables that couldn't be enabled.
+   *         The return HTDs are read-only.
    * @see #enableTables(java.util.regex.Pattern)
    * @see #enableTable(org.apache.hadoop.hbase.TableName)
    */
@@ -386,6 +390,8 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param pattern The pattern to match table names against
    * @throws IOException
+   * @return Table descriptors for tables that couldn't be enabled.
+   *         The return HTDs are read-only.
    */
   HTableDescriptor[] enableTables(Pattern pattern) throws IOException;
 
@@ -422,6 +428,7 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be disabled
+   *         The return htds are read-only
    * @throws IOException
    * @see #disableTables(java.util.regex.Pattern)
    * @see #disableTable(org.apache.hadoop.hbase.TableName)
@@ -436,6 +443,7 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be disabled
+   *         The return htds are read-only
    * @throws IOException
    */
   HTableDescriptor[] disableTables(Pattern pattern) throws IOException;
@@ -1166,7 +1174,7 @@ public interface Admin extends Abortable, Closeable {
    * Get list of table descriptors by namespace
    *
    * @param name namespace name
-   * @return A descriptor
+   * @return HTD[] the read-only tableDescriptors
    * @throws IOException
    */
   HTableDescriptor[] listTableDescriptorsByNamespace(final String name)
@@ -1199,7 +1207,7 @@ public interface Admin extends Abortable, Closeable {
    * Get tableDescriptors
    *
    * @param tableNames List of table names
-   * @return HTD[] the tableDescriptor
+   * @return HTD[] the read-only tableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
@@ -1209,7 +1217,7 @@ public interface Admin extends Abortable, Closeable {
    * Get tableDescriptors
    *
    * @param names List of table names
-   * @return HTD[] the tableDescriptor
+   * @return HTD[] the read-only tableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   HTableDescriptor[] getTableDescriptors(List<String> names)

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 b7c60dd..3343c7a 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
@@ -25,7 +25,6 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -57,27 +56,27 @@ public interface AsyncAdmin {
 
   /**
    * List all the userspace tables.
-   * @return - returns an array of HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @return - returns an array of TableDescriptors wrapped by a {@link CompletableFuture}.
    * @see #listTables(Pattern, boolean)
    */
-  CompletableFuture<HTableDescriptor[]> listTables();
+  CompletableFuture<TableDescriptor[]> listTables();
 
   /**
    * List all the tables matching the given pattern.
    * @param regex The regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @return - returns an array of TableDescriptors wrapped by a {@link CompletableFuture}.
    * @see #listTables(Pattern, boolean)
    */
-  CompletableFuture<HTableDescriptor[]> listTables(String regex, boolean includeSysTables);
+  CompletableFuture<TableDescriptor[]> listTables(String regex, boolean includeSysTables);
 
   /**
    * List all the tables matching the given pattern.
    * @param pattern The compiled regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @return - returns an array of TableDescriptors wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<HTableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables);
+  CompletableFuture<TableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables);
 
   /**
    * List all of the names of userspace tables.
@@ -107,15 +106,15 @@ public interface AsyncAdmin {
   /**
    * Method for getting the tableDescriptor
    * @param tableName as a {@link TableName}
-   * @return the tableDescriptor wrapped by a {@link CompletableFuture}.
+   * @return the read-only tableDescriptor wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<HTableDescriptor> getTableDescriptor(final TableName tableName);
+  CompletableFuture<TableDescriptor> getTableDescriptor(final TableName tableName);
 
   /**
    * Creates a new table.
    * @param desc table descriptor for table
    */
-  CompletableFuture<Void> createTable(HTableDescriptor desc);
+  CompletableFuture<Void> createTable(TableDescriptor desc);
 
   /**
    * Creates a new table with the specified number of regions. The start key specified will become
@@ -128,7 +127,7 @@ public interface AsyncAdmin {
    * @param endKey end of key range
    * @param numRegions the total number of regions to create
    */
-  CompletableFuture<Void> createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey,
+  CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey,
       int numRegions);
 
   /**
@@ -138,7 +137,7 @@ public interface AsyncAdmin {
    * @param desc table descriptor for table
    * @param splitKeys array of split keys for the initial regions of the table
    */
-  CompletableFuture<Void> createTable(final HTableDescriptor desc, byte[][] splitKeys);
+  CompletableFuture<Void> createTable(final TableDescriptor desc, byte[][] splitKeys);
 
   /**
    * Deletes a table.
@@ -153,9 +152,9 @@ public interface AsyncAdmin {
    * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be deleted. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> deleteTables(String regex);
+  CompletableFuture<TableDescriptor[]> deleteTables(String regex);
 
   /**
    * Delete tables matching the passed in pattern and wait on completion. Warning: Use this method
@@ -164,9 +163,9 @@ public interface AsyncAdmin {
    * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> deleteTables(Pattern pattern);
+  CompletableFuture<TableDescriptor[]> deleteTables(Pattern pattern);
 
   /**
    * Truncate a table.
@@ -187,9 +186,9 @@ public interface AsyncAdmin {
    * {@link #enableTable(TableName)}
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be enabled. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> enableTables(String regex);
+  CompletableFuture<TableDescriptor[]> enableTables(String regex);
 
   /**
    * Enable tables matching the passed in pattern. Warning: Use this method carefully, there is no
@@ -197,9 +196,9 @@ public interface AsyncAdmin {
    * {@link #enableTable(TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be enabled. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> enableTables(Pattern pattern);
+  CompletableFuture<TableDescriptor[]> enableTables(Pattern pattern);
 
   /**
    * Disable a table. The table has to be in enabled state for it to be disabled.
@@ -213,9 +212,9 @@ public interface AsyncAdmin {
    * {@link #disableTable(TableName)}
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be disabled. The return value will be wrapped by a
-   *         {@link CompletableFuture}.
+   *         {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> disableTables(String regex);
+  CompletableFuture<TableDescriptor[]> disableTables(String regex);
 
   /**
    * Disable tables matching the passed in pattern. Warning: Use this method carefully, there is no
@@ -223,9 +222,9 @@ public interface AsyncAdmin {
    * {@link #disableTable(TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be disabled. The return value will be wrapped by a
-   *         {@link CompletableFuture}.
+   *         {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> disableTables(Pattern pattern);
+  CompletableFuture<TableDescriptor[]> disableTables(Pattern pattern);
 
   /**
    * @param tableName name of table to check

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 54e1e8b..a54cc7a 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
@@ -44,7 +44,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
 import org.apache.hadoop.hbase.NotServingRegionException;
@@ -293,10 +292,10 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
     CompletableFuture<Void> operate(TableName table);
   }
 
-  private CompletableFuture<HTableDescriptor[]> batchTableOperations(Pattern pattern,
+  private CompletableFuture<TableDescriptor[]> batchTableOperations(Pattern pattern,
       TableOperator operator, String operationType) {
-    CompletableFuture<HTableDescriptor[]> future = new CompletableFuture<>();
-    List<HTableDescriptor> failed = new LinkedList<>();
+    CompletableFuture<TableDescriptor[]> future = new CompletableFuture<>();
+    List<TableDescriptor> failed = new LinkedList<>();
     listTables(pattern, false).whenComplete(
       (tables, error) -> {
         if (error != null) {
@@ -311,7 +310,7 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
               }
             })).<CompletableFuture> toArray(size -> new CompletableFuture[size]);
         CompletableFuture.allOf(futures).thenAccept((v) -> {
-          future.complete(failed.toArray(new HTableDescriptor[failed.size()]));
+          future.complete(failed.toArray(new TableDescriptor[failed.size()]));
         });
       });
     return future;
@@ -328,25 +327,25 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> listTables() {
+  public CompletableFuture<TableDescriptor[]> listTables() {
     return listTables((Pattern) null, false);
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> listTables(String regex, boolean includeSysTables) {
+  public CompletableFuture<TableDescriptor[]> listTables(String regex, boolean includeSysTables) {
     return listTables(Pattern.compile(regex), false);
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables) {
+  public CompletableFuture<TableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables) {
     return this
-        .<HTableDescriptor[]>newMasterCaller()
+        .<TableDescriptor[]>newMasterCaller()
         .action(
           (controller, stub) -> this
-              .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, HTableDescriptor[]> call(
+              .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, TableDescriptor[]> call(
                 controller, stub, RequestConverter.buildGetTableDescriptorsRequest(pattern,
                   includeSysTables), (s, c, req, done) -> s.getTableDescriptors(c, req, done), (
-                    resp) -> ProtobufUtil.getHTableDescriptorArray(resp))).call();
+                    resp) -> ProtobufUtil.getTableDescriptorArray(resp))).call();
   }
 
   @Override
@@ -372,8 +371,8 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor> getTableDescriptor(TableName tableName) {
-    CompletableFuture<HTableDescriptor> future = new CompletableFuture<>();
+  public CompletableFuture<TableDescriptor> getTableDescriptor(TableName tableName) {
+    CompletableFuture<TableDescriptor> future = new CompletableFuture<>();
     this.<List<TableSchema>> newMasterCaller()
         .action(
           (controller, stub) -> this
@@ -386,7 +385,7 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
             return;
           }
           if (!tableSchemas.isEmpty()) {
-            future.complete(ProtobufUtil.convertToHTableDesc(tableSchemas.get(0)));
+            future.complete(ProtobufUtil.convertToTableDesc(tableSchemas.get(0)));
           } else {
             future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
           }
@@ -395,12 +394,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> createTable(HTableDescriptor desc) {
+  public CompletableFuture<Void> createTable(TableDescriptor desc) {
     return createTable(desc, null);
   }
 
   @Override
-  public CompletableFuture<Void> createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey,
+  public CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey,
       int numRegions) {
     try {
       return createTable(desc, getSplitKeys(startKey, endKey, numRegions));
@@ -410,7 +409,7 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> createTable(HTableDescriptor desc, byte[][] splitKeys) {
+  public CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys) {
     if (desc.getTableName() == null) {
       return failedFuture(new IllegalArgumentException("TableName cannot be null"));
     }
@@ -447,12 +446,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> deleteTables(String regex) {
+  public CompletableFuture<TableDescriptor[]> deleteTables(String regex) {
     return deleteTables(Pattern.compile(regex));
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> deleteTables(Pattern pattern) {
+  public CompletableFuture<TableDescriptor[]> deleteTables(Pattern pattern) {
     return batchTableOperations(pattern, (table) -> deleteTable(table), "DELETE");
   }
 
@@ -473,12 +472,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> enableTables(String regex) {
+  public CompletableFuture<TableDescriptor[]> enableTables(String regex) {
     return enableTables(Pattern.compile(regex));
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> enableTables(Pattern pattern) {
+  public CompletableFuture<TableDescriptor[]> enableTables(Pattern pattern) {
     return batchTableOperations(pattern, (table) -> enableTable(table), "ENABLE");
   }
 
@@ -491,12 +490,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> disableTables(String regex) {
+  public CompletableFuture<TableDescriptor[]> disableTables(String regex) {
     return disableTables(Pattern.compile(regex));
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> disableTables(Pattern pattern) {
+  public CompletableFuture<TableDescriptor[]> disableTables(Pattern pattern) {
     return batchTableOperations(pattern, (table) -> disableTable(table), "DISABLE");
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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 7e79c20..14af586 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
@@ -441,7 +441,7 @@ public class HBaseAdmin implements Admin {
       }
     }, rpcCallerFactory, operationTimeout, rpcTimeout);
     if (htd != null) {
-      return htd;
+      return new ImmutableHTableDescriptor(htd);
     }
     throw new TableNotFoundException(tableName.getNameAsString());
   }
@@ -532,7 +532,7 @@ public class HBaseAdmin implements Admin {
       super(admin, desc.getTableName(),
               (response != null && response.hasProcId()) ? response.getProcId() : null);
       this.splitKeys = splitKeys;
-      this.desc = desc;
+      this.desc = new ImmutableHTableDescriptor(desc);
     }
 
     @Override
@@ -2138,8 +2138,7 @@ public class HBaseAdmin implements Admin {
                 .build()).getTableSchemaList();
         HTableDescriptor[] res = new HTableDescriptor[list.size()];
         for(int i=0; i < list.size(); i++) {
-
-          res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
+          res[i] = new ImmutableHTableDescriptor(ProtobufUtil.convertToHTableDesc(list.get(i)));
         }
         return res;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 3bdbed5..e89d4ed 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -266,7 +266,7 @@ public class HTable implements Table {
     HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
       rpcControllerFactory, operationTimeout, readRpcTimeout);
     if (htd != null) {
-      return new UnmodifyableHTableDescriptor(htd);
+      return new ImmutableHTableDescriptor(htd);
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
new file mode 100644
index 0000000..89d1291
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
@@ -0,0 +1,79 @@
+/**
+ *
+ * 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.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Read-only table descriptor.
+ */
+@Deprecated // deprecated for hbase 2.0, remove for hbase 3.0. see HTableDescriptor.
+@InterfaceAudience.Public
+public class ImmutableHTableDescriptor extends HTableDescriptor {
+
+  /*
+   * Create an unmodifyable copy of an HTableDescriptor
+   * @param desc
+   */
+  public ImmutableHTableDescriptor(final HTableDescriptor desc) {
+    super(new UnmodifyableTableDescriptor(desc));
+  }
+
+  @Deprecated // deprecated for hbase 2.0, remove for hbase 3.0. see HTableDescriptor.
+  private static class UnmodifyableTableDescriptor extends ModifyableTableDescriptor {
+
+    UnmodifyableTableDescriptor(final TableDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    protected ModifyableTableDescriptor setFamily(HColumnDescriptor family) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public HColumnDescriptor removeFamily(final byte[] column) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public ModifyableTableDescriptor setValue(final Bytes key, final Bytes value) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public void remove(Bytes key) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public ModifyableTableDescriptor setConfiguration(String key, String value) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public void removeConfiguration(final String key) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+  }
+}


[19/40] hbase git commit: HBASE-13288 Fix naming of parameter in Delete constructor

Posted by sy...@apache.org.
HBASE-13288 Fix naming of parameter in Delete constructor

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/87f2bb57
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/87f2bb57
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/87f2bb57

Branch: refs/heads/hbase-12439
Commit: 87f2bb5796bd2a05f2c9db559ddd13a33fc80e36
Parents: 49cba2c
Author: Ashish Singhi <as...@huawei.com>
Authored: Thu Mar 19 22:04:25 2015 +0530
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Apr 21 11:09:44 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Delete.java    | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/87f2bb57/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 278ea58..0b3769d 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
@@ -108,12 +108,12 @@ public class Delete extends Mutation implements Comparable<Row> {
    *
    * This timestamp is ONLY used for a delete row operation.  If specifying
    * families or columns, you must specify each timestamp individually.
-   * @param rowArray We make a local copy of this passed in row.
+   * @param row We make a local copy of this passed in row.
    * @param rowOffset
    * @param rowLength
    */
-  public Delete(final byte [] rowArray, final int rowOffset, final int rowLength) {
-    this(rowArray, rowOffset, rowLength, HConstants.LATEST_TIMESTAMP);
+  public Delete(final byte[] row, final int rowOffset, final int rowLength) {
+    this(row, rowOffset, rowLength, HConstants.LATEST_TIMESTAMP);
   }
 
   /**
@@ -125,15 +125,15 @@ public class Delete extends Mutation implements Comparable<Row> {
    *
    * This timestamp is ONLY used for a delete row operation.  If specifying
    * families or columns, you must specify each timestamp individually.
-   * @param rowArray We make a local copy of this passed in row.
+   * @param row We make a local copy of this passed in row.
    * @param rowOffset
    * @param rowLength
-   * @param ts maximum version timestamp (only for delete row)
+   * @param timestamp maximum version timestamp (only for delete row)
    */
-  public Delete(final byte [] rowArray, final int rowOffset, final int rowLength, long ts) {
-    checkRow(rowArray, rowOffset, rowLength);
-    this.row = Bytes.copy(rowArray, rowOffset, rowLength);
-    setTimestamp(ts);
+  public Delete(final byte[] row, final int rowOffset, final int rowLength, long timestamp) {
+    checkRow(row, rowOffset, rowLength);
+    this.row = Bytes.copy(row, rowOffset, rowLength);
+    setTimestamp(timestamp);
   }
 
   /**


[29/40] hbase git commit: HBASE-17933: [hbase-spark] Support Java api for bulkload

Posted by sy...@apache.org.
HBASE-17933: [hbase-spark] Support Java api for bulkload

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/49f707fb
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/49f707fb
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/49f707fb

Branch: refs/heads/hbase-12439
Commit: 49f707fba7c6a9f0210f387e31d1be9f108991f8
Parents: 9a1aff4
Author: Yi Liang <ea...@gmail.com>
Authored: Fri Apr 21 18:10:03 2017 -0700
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Apr 24 11:48:29 2017 -0500

----------------------------------------------------------------------
 .../hbasecontext/JavaHBaseBulkLoadExample.java  | 102 ++++++++++
 .../hbase/spark/FamiliesQualifiersValues.scala  |  12 +-
 .../hadoop/hbase/spark/JavaHBaseContext.scala   |  68 ++++++-
 .../hbase/spark/TestJavaHBaseContext.java       | 201 ++++++++++++++++++-
 4 files changed, 371 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/49f707fb/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java
new file mode 100644
index 0000000..040546d
--- /dev/null
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.spark.example.hbasecontext;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.spark.FamilyHFileWriteOptions;
+import org.apache.hadoop.hbase.spark.JavaHBaseContext;
+import org.apache.hadoop.hbase.spark.KeyFamilyQualifier;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.function.Function;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * Run this example using command below:
+ *
+ *  SPARK_HOME/bin/spark-submit --master local[2] --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample
+ *  path/to/hbase-spark.jar {path/to/output/HFiles}
+ *
+ * This example will output put hfiles in {path/to/output/HFiles}, and user can run
+ * 'hbase org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles' to load the HFiles into table to verify this example.
+ */
+final public class JavaHBaseBulkLoadExample {
+  private JavaHBaseBulkLoadExample() {}
+
+  public static void main(String[] args) {
+    if (args.length < 1) {
+      System.out.println("JavaHBaseBulkLoadExample  " + "{outputPath}");
+      return;
+    }
+
+    String tableName = "bulkload-table-test";
+    String columnFamily1 = "f1";
+    String columnFamily2 = "f2";
+
+    SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkLoadExample " + tableName);
+    JavaSparkContext jsc = new JavaSparkContext(sparkConf);
+
+    try {
+      List<String> list= new ArrayList<String>();
+      // row1
+      list.add("1," + columnFamily1 + ",b,1");
+      // row3
+      list.add("3," + columnFamily1 + ",a,2");
+      list.add("3," + columnFamily1 + ",b,1");
+      list.add("3," + columnFamily2 + ",a,1");
+      /* row2 */
+      list.add("2," + columnFamily2 + ",a,3");
+      list.add("2," + columnFamily2 + ",b,3");
+
+      JavaRDD<String> rdd = jsc.parallelize(list);
+
+      Configuration conf = HBaseConfiguration.create();
+      JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
+
+
+
+      hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName),new BulkLoadFunction(), args[0],
+          new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
+    } finally {
+      jsc.stop();
+    }
+  }
+
+  public static class BulkLoadFunction implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
+
+    @Override
+    public Pair<KeyFamilyQualifier, byte[]> call(String v1) throws Exception {
+      if (v1 == null)
+        return null;
+      String[] strs = v1.split(",");
+      if(strs.length != 4)
+        return null;
+      KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]),
+          Bytes.toBytes(strs[2]));
+      return new Pair(kfq, Bytes.toBytes(strs[3]));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/49f707fb/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 92bb3b7..7733802 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
@@ -55,4 +55,14 @@ class FamiliesQualifiersValues extends Serializable {
 
     qualifierValues.put(new ByteArrayWrapper(qualifier), value)
   }
-}
+
+  /**
+    * A wrapper for "+=" method above, can be used by Java
+    * @param family    HBase column family
+    * @param qualifier HBase column qualifier
+    * @param value     HBase cell value
+    */
+  def add(family: Array[Byte], qualifier: Array[Byte], value: Array[Byte]): Unit = {
+    this += (family, qualifier, value)
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/49f707fb/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 253b386..57029f3 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
@@ -17,9 +17,12 @@
 
 package org.apache.hadoop.hbase.spark
 
+import java.util.Map
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hbase.TableName
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Pair
+import org.apache.hadoop.hbase.classification.InterfaceAudience
 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}
@@ -268,7 +271,6 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
    * generates a new DStream based on Gets and the results
    * they bring back from HBase
    *
-
    * @param tableName     The name of the table to get from
    * @param batchSize     The number of gets to be batched together
    * @param javaDStream   Original DStream with data to iterate over
@@ -292,6 +294,67 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
   }
 
   /**
+    * A simple abstraction over the HBaseContext.bulkLoad method.
+    * It allow addition support for a user to take a JavaRDD and
+    * convert into new JavaRDD[Pair] based on MapFunction,
+    * and HFiles will be generated in stagingDir for bulk load
+    *
+    * @param javaRdd                        The javaRDD we are bulk loading from
+    * @param tableName                      The HBase table we are loading into
+    * @param mapFunc                        A Function that will convert a value in JavaRDD
+    *                                       to Pair(KeyFamilyQualifier, Array[Byte])
+    * @param stagingDir                     The location on the FileSystem to bulk load into
+    * @param familyHFileWriteOptionsMap     Options that will define how the HFile for a
+    *                                       column family is written
+    * @param compactionExclude              Compaction excluded for the HFiles
+    * @param maxSize                        Max size for the HFiles before they roll
+    */
+  def bulkLoad[T](javaRdd: JavaRDD[T],
+                  tableName: TableName,
+                  mapFunc : Function[T, Pair[KeyFamilyQualifier, Array[Byte]]],
+                  stagingDir: String,
+                  familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions],
+                  compactionExclude: Boolean,
+                  maxSize: Long):
+  Unit = {
+    hbaseContext.bulkLoad[Pair[KeyFamilyQualifier, Array[Byte]]](javaRdd.map(mapFunc).rdd, tableName, t => {
+      val keyFamilyQualifier = t.getFirst
+      val value = t.getSecond
+      Seq((keyFamilyQualifier, value)).iterator
+    }, stagingDir, familyHFileWriteOptionsMap, compactionExclude, maxSize)
+  }
+
+  /**
+    * A simple abstraction over the HBaseContext.bulkLoadThinRows method.
+    * It allow addition support for a user to take a JavaRDD and
+    * convert into new JavaRDD[Pair] based on MapFunction,
+    * and HFiles will be generated in stagingDir for bulk load
+    *
+    * @param javaRdd                        The javaRDD we are bulk loading from
+    * @param tableName                      The HBase table we are loading into
+    * @param mapFunc                        A Function that will convert a value in JavaRDD
+    *                                       to Pair(ByteArrayWrapper, FamiliesQualifiersValues)
+    * @param stagingDir                     The location on the FileSystem to bulk load into
+    * @param familyHFileWriteOptionsMap     Options that will define how the HFile for a
+    *                                       column family is written
+    * @param compactionExclude              Compaction excluded for the HFiles
+    * @param maxSize                        Max size for the HFiles before they roll
+    */
+  def bulkLoadThinRows[T](javaRdd: JavaRDD[T],
+                       tableName: TableName,
+                       mapFunc : Function[T, Pair[ByteArrayWrapper, FamiliesQualifiersValues]],
+                       stagingDir: String,
+                       familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions],
+                       compactionExclude: Boolean,
+                       maxSize: Long):
+  Unit = {
+    hbaseContext.bulkLoadThinRows[Pair[ByteArrayWrapper, FamiliesQualifiersValues]](javaRdd.map(mapFunc).rdd,
+      tableName, t => {
+      (t.getFirst, t.getSecond)
+    }, stagingDir, familyHFileWriteOptionsMap, compactionExclude, maxSize)
+  }
+
+  /**
    * This function will use the native HBase TableInputFormat with the
    * given scan object to generate a new JavaRDD
    *
@@ -341,4 +404,5 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
    */
   private[spark]
   def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]]
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/49f707fb/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java b/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
index da6b724..c3f1bcb 100644
--- a/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
+++ b/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java
@@ -19,16 +19,22 @@ package org.apache.hadoop.hbase.spark;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
-
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.HashMap;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -38,17 +44,24 @@ 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.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkDeleteExample;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.spark.api.java.*;
+
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.api.java.function.Function;
-import org.junit.*;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import scala.Tuple2;
-
 import com.google.common.io.Files;
 
 @Category({MiscTests.class, MediumTests.class})
@@ -58,19 +71,22 @@ public class TestJavaHBaseContext implements Serializable {
   protected static final Log LOG = LogFactory.getLog(TestJavaHBaseContext.class);
 
 
+
   byte[] tableName = Bytes.toBytes("t1");
   byte[] columnFamily = Bytes.toBytes("c");
+  byte[] columnFamily1 = Bytes.toBytes("d");
   String columnFamilyStr = Bytes.toString(columnFamily);
+  String columnFamilyStr1 = Bytes.toString(columnFamily1);
+
 
   @Before
   public void setUp() {
     jsc = new JavaSparkContext("local", "JavaHBaseContextSuite");
-    jsc.addJar("spark.jar");
 
     File tempDir = Files.createTempDir();
     tempDir.deleteOnExit();
 
-    htu = HBaseTestingUtility.createLocalHTU();
+    htu = new HBaseTestingUtility();
     try {
       LOG.info("cleaning up test dir");
 
@@ -91,7 +107,7 @@ public class TestJavaHBaseContext implements Serializable {
 
       LOG.info(" - creating table " + Bytes.toString(tableName));
       htu.createTable(TableName.valueOf(tableName),
-              columnFamily);
+          new byte[][]{columnFamily, columnFamily1});
       LOG.info(" - created table");
     } catch (Exception e1) {
       throw new RuntimeException(e1);
@@ -278,6 +294,173 @@ public class TestJavaHBaseContext implements Serializable {
     Assert.assertEquals(stringJavaRDD.count(), 5);
   }
 
+  @Test
+  public void testBulkLoad() throws Exception {
+
+    Path output = htu.getDataTestDir("testBulkLoad");
+    // Add cell as String: "row,falmily,qualifier,value"
+    List<String> list= new ArrayList<String>();
+    // row1
+    list.add("1," + columnFamilyStr + ",b,1");
+    // row3
+    list.add("3," + columnFamilyStr + ",a,2");
+    list.add("3," + columnFamilyStr + ",b,1");
+    list.add("3," + columnFamilyStr1 + ",a,1");
+    //row2
+    list.add("2," + columnFamilyStr + ",a,3");
+    list.add("2," + columnFamilyStr + ",b,3");
+
+    JavaRDD<String> rdd = jsc.parallelize(list);
+
+    Configuration conf = htu.getConfiguration();
+    JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
+
+
+
+    hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(), output.toUri().getPath(),
+        new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
+
+    try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) {
+      Table table = conn.getTable(TableName.valueOf(tableName));
+      // Do bulk load
+      LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf);
+      load.doBulkLoad(output, admin, table, conn.getRegionLocator(TableName.valueOf(tableName)));
+
+
+
+      // Check row1
+      List<Cell> cell1 = table.get(new Get(Bytes.toBytes("1"))).listCells();
+      Assert.assertEquals(cell1.size(), 1);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell1.get(0))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell1.get(0))), "b");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell1.get(0))), "1");
+
+      // Check row3
+      List<Cell> cell3 = table.get(new Get(Bytes.toBytes("3"))).listCells();
+      Assert.assertEquals(cell3.size(), 3);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(0))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(0))), "a");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(0))), "2");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(1))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(1))), "b");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(1))), "1");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(2))), columnFamilyStr1);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(2))), "a");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(2))), "1");
+
+      // Check row2
+      List<Cell> cell2 = table.get(new Get(Bytes.toBytes("2"))).listCells();
+      Assert.assertEquals(cell2.size(), 2);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(0))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(0))), "a");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(0))), "3");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(1))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(1))), "b");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(1))), "3");
+    }
+  }
+
+  @Test
+  public void testBulkLoadThinRows() throws Exception {
+    Path output = htu.getDataTestDir("testBulkLoadThinRows");
+    // because of the limitation of scala bulkLoadThinRows API
+    // we need to provide data as <row, all cells in that row>
+    List<List<String>> list= new ArrayList<List<String>>();
+    // row1
+    List<String> list1 = new ArrayList<String>();
+    list1.add("1," + columnFamilyStr + ",b,1");
+    list.add(list1);
+    // row3
+    List<String> list3 = new ArrayList<String>();
+    list3.add("3," + columnFamilyStr + ",a,2");
+    list3.add("3," + columnFamilyStr + ",b,1");
+    list3.add("3," + columnFamilyStr1 + ",a,1");
+    list.add(list3);
+    //row2
+    List<String> list2 = new ArrayList<String>();
+    list2.add("2," + columnFamilyStr + ",a,3");
+    list2.add("2," + columnFamilyStr + ",b,3");
+    list.add(list2);
+
+    JavaRDD<List<String>> rdd = jsc.parallelize(list);
+
+    Configuration conf = htu.getConfiguration();
+    JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
+
+    hbaseContext.bulkLoadThinRows(rdd, TableName.valueOf(tableName), new BulkLoadThinRowsFunction(), output.toString(),
+        new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
+
+
+    try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) {
+      Table table = conn.getTable(TableName.valueOf(tableName));
+      // Do bulk load
+      LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf);
+      load.doBulkLoad(output, admin, table, conn.getRegionLocator(TableName.valueOf(tableName)));
+
+      // Check row1
+      List<Cell> cell1 = table.get(new Get(Bytes.toBytes("1"))).listCells();
+      Assert.assertEquals(cell1.size(), 1);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell1.get(0))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell1.get(0))), "b");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell1.get(0))), "1");
+
+      // Check row3
+      List<Cell> cell3 = table.get(new Get(Bytes.toBytes("3"))).listCells();
+      Assert.assertEquals(cell3.size(), 3);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(0))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(0))), "a");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(0))), "2");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(1))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(1))), "b");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(1))), "1");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(2))), columnFamilyStr1);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(2))), "a");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(2))), "1");
+
+      // Check row2
+      List<Cell> cell2 = table.get(new Get(Bytes.toBytes("2"))).listCells();
+      Assert.assertEquals(cell2.size(), 2);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(0))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(0))), "a");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(0))), "3");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(1))), columnFamilyStr);
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(1))), "b");
+      Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(1))), "3");
+    }
+
+  }
+  public static class BulkLoadFunction implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
+
+    @Override public Pair<KeyFamilyQualifier, byte[]> call(String v1) throws Exception {
+      if (v1 == null)
+        return null;
+      String[] strs = v1.split(",");
+      if(strs.length != 4)
+        return null;
+      KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]),
+          Bytes.toBytes(strs[2]));
+      return new Pair(kfq, Bytes.toBytes(strs[3]));
+    }
+  }
+
+  public static class BulkLoadThinRowsFunction implements Function<List<String>, Pair<ByteArrayWrapper, FamiliesQualifiersValues>> {
+
+    @Override public Pair<ByteArrayWrapper, FamiliesQualifiersValues> call(List<String> list) throws Exception {
+      if (list == null)
+        return null;
+      ByteArrayWrapper rowKey = null;
+      FamiliesQualifiersValues fqv = new FamiliesQualifiersValues();
+      for (String cell : list) {
+        String[] strs = cell.split(",");
+        if (rowKey == null) {
+          rowKey = new ByteArrayWrapper(Bytes.toBytes(strs[0]));
+        }
+        fqv.add(Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2]), Bytes.toBytes(strs[3]));
+      }
+      return new Pair(rowKey, fqv);
+    }
+  }
+
   public static class GetFunction implements Function<byte[], Get> {
 
     private static final long serialVersionUID = 1L;
@@ -335,4 +518,4 @@ public class TestJavaHBaseContext implements Serializable {
     }
   }
 
-}
\ No newline at end of file
+}


[25/40] hbase git commit: HBASE-16314 Retry on table snapshot failure during full backup (Vladimir Rodionov)

Posted by sy...@apache.org.
HBASE-16314 Retry on table snapshot failure during full backup (Vladimir Rodionov)


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

Branch: refs/heads/hbase-12439
Commit: e95cf479c7615ae160a6ba963cc7689f3b440efd
Parents: a3b6f4a
Author: tedyu <yu...@gmail.com>
Authored: Fri Apr 21 16:15:07 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Apr 21 16:15:07 2017 -0700

----------------------------------------------------------------------
 .../hbase/backup/BackupRestoreConstants.java    | 10 ++++++
 .../backup/impl/FullTableBackupClient.java      | 36 ++++++++++++++++++--
 2 files changed, 44 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e95cf479/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
index e46904b..d1ab246 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -37,6 +37,16 @@ public interface BackupRestoreConstants {
   public final static int BACKUP_SYSTEM_TTL_DEFAULT = HConstants.FOREVER;
   public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable";
   public final static boolean BACKUP_ENABLE_DEFAULT = false;
+
+
+  public static final String BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.attempts.max";
+  public static final int DEFAULT_BACKUP_MAX_ATTEMPTS = 10;
+
+  public static final String BACKUP_ATTEMPTS_PAUSE_MS_KEY = "hbase.backup.attempts.pause.ms";
+  public static final int DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS = 10000;
+
+
+
   /*
    *  Drivers option list
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/e95cf479/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
index 77d1184..ee7a841 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.hbase.backup.impl;
 
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_ATTEMPTS_PAUSE_MS_KEY;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_MAX_ATTEMPTS;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
@@ -148,8 +153,7 @@ public class FullTableBackupClient extends TableBackupClient {
             "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
                 + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString();
 
-        admin.snapshot(snapshotName, tableName);
-
+        snapshotTable(admin, tableName, snapshotName);
         backupInfo.setSnapshotName(tableName, snapshotName);
       }
 
@@ -186,4 +190,32 @@ public class FullTableBackupClient extends TableBackupClient {
 
   }
 
+  private void snapshotTable(Admin admin, TableName tableName, String snapshotName)
+      throws IOException {
+
+    int maxAttempts =
+        conf.getInt(BACKUP_MAX_ATTEMPTS_KEY, DEFAULT_BACKUP_MAX_ATTEMPTS);
+    int pause =
+        conf.getInt(BACKUP_ATTEMPTS_PAUSE_MS_KEY, DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS);
+    int attempts = 0;
+
+    while (attempts++ < maxAttempts) {
+      try {
+        admin.snapshot(snapshotName, tableName);
+        return;
+      } catch (IOException ee) {
+        LOG.warn("Snapshot attempt " + attempts + " failed for table " + tableName
+            + ", sleeping for " + pause + "ms", ee);
+        if (attempts < maxAttempts) {
+          try {
+            Thread.sleep(pause);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            break;
+          }
+        }
+      }
+    }
+    throw new IOException("Failed to snapshot table "+ tableName);
+  }
 }


[17/40] hbase git commit: HBASE-17915 Implement async replication admin methods

Posted by sy...@apache.org.
HBASE-17915 Implement async replication admin methods


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

Branch: refs/heads/hbase-12439
Commit: 40cc666ac984e846a8c7105b771ce6bec90c4ad3
Parents: b1ef8dd
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Apr 20 18:13:03 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu Apr 20 18:13:03 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  92 ++++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    | 249 +++++++++--
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  58 +--
 .../replication/ReplicationSerDeHelper.java     |  67 +++
 .../hadoop/hbase/client/TestAsyncAdminBase.java |   2 +-
 .../client/TestAsyncReplicationAdminApi.java    | 416 +++++++++++++++++++
 6 files changed, 802 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40cc666a/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 270f28f..5d2955f 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hbase.client;
 
 import java.util.List;
+import java.util.Collection;
+import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.regex.Pattern;
 
@@ -30,6 +32,9 @@ 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.client.replication.TableCFs;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.util.Pair;
 
 /**
@@ -481,4 +486,91 @@ public interface AsyncAdmin {
    * @return the QuotaSetting list, which wrapped by a CompletableFuture.
    */
   CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter);
+
+  /**
+   * Add a new replication peer for replicating data to slave cluster
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig configuration for the replication slave cluster
+   */
+  CompletableFuture<Void> addReplicationPeer(final String peerId,
+      final ReplicationPeerConfig peerConfig);
+
+  /**
+   * Remove a peer and stop the replication
+   * @param peerId a short name that identifies the peer
+   */
+  CompletableFuture<Void> removeReplicationPeer(final String peerId);
+
+  /**
+   * Restart the replication stream to the specified peer
+   * @param peerId a short name that identifies the peer
+   */
+  CompletableFuture<Void> enableReplicationPeer(final String peerId);
+
+  /**
+   * Stop the replication stream to the specified peer
+   * @param peerId a short name that identifies the peer
+   */
+  CompletableFuture<Void> disableReplicationPeer(final String peerId);
+
+  /**
+   * Returns the configured ReplicationPeerConfig for the specified peer
+   * @param peerId a short name that identifies the peer
+   * @return ReplicationPeerConfig for the peer wrapped by a {@link CompletableFuture}.
+   */
+  CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(final String peerId);
+
+  /**
+   * Update the peerConfig for the specified peer
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig new config for the peer
+   */
+  CompletableFuture<Void> updateReplicationPeerConfig(final String peerId,
+      final ReplicationPeerConfig peerConfig);
+
+  /**
+   * Append the replicable table-cf config of the specified peer
+   * @param id a short that identifies the cluster
+   * @param tableCfs A map from tableName to column family names
+   */
+  CompletableFuture<Void> appendReplicationPeerTableCFs(String id,
+      Map<TableName, ? extends Collection<String>> tableCfs);
+
+  /**
+   * Remove some table-cfs from config of the specified peer
+   * @param id a short name that identifies the cluster
+   * @param tableCfs A map from tableName to column family names
+   */
+  CompletableFuture<Void> removeReplicationPeerTableCFs(String id,
+      Map<TableName, ? extends Collection<String>> tableCfs);
+
+  /**
+   * Return a list of replication peers.
+   * @return a list of replication peers description. The return value will be wrapped by a
+   *         {@link CompletableFuture}.
+   */
+  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers();
+
+  /**
+   * Return a list of replication peers.
+   * @param regex The regular expression to match peer id
+   * @return a list of replication peers description. The return value will be wrapped by a
+   *         {@link CompletableFuture}.
+   */
+  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(String regex);
+
+  /**
+   * Return a list of replication peers.
+   * @param pattern The compiled regular expression to match peer id
+   * @return a list of replication peers description. The return value will be wrapped by a
+   *         {@link CompletableFuture}.
+   */
+  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern);
+
+  /**
+   * Find all table and column families that are replicated from this cluster
+   * @return the replicated table-cfs list of this cluster. The return value will be wrapped by a
+   *         {@link CompletableFuture}.
+   */
+  CompletableFuture<List<TableCFs>> listReplicatedTableCFs();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40cc666a/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 180cd19..eae4089 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
@@ -22,16 +22,21 @@ import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -54,11 +59,16 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
 import org.apache.hadoop.hbase.client.Scan.ReadType;
+import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
+import org.apache.hadoop.hbase.client.replication.TableCFs;
 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.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 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;
@@ -121,6 +131,20 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
@@ -1155,42 +1179,209 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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();
+  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);
+    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 onError(Throwable error) {
+            future.completeExceptionally(error);
+          }
+
+          @Override
+          public void onComplete() {
+            future.complete(settings);
+          }
+        });
+    return future;
+  }
+
+  public CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig) {
+    return this
+        .<Void> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<AddReplicationPeerRequest, AddReplicationPeerResponse, Void> call(controller, stub,
+                RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig), (s, c, req,
+                    done) -> s.addReplicationPeer(c, req, done), (resp) -> null)).call();
+  }
+
+  @Override
+  public CompletableFuture<Void> removeReplicationPeer(String peerId) {
+    return this
+        .<Void> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<RemoveReplicationPeerRequest, RemoveReplicationPeerResponse, Void> call(controller,
+                stub, RequestConverter.buildRemoveReplicationPeerRequest(peerId),
+                (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> null)).call();
+  }
+
+  @Override
+  public CompletableFuture<Void> enableReplicationPeer(String peerId) {
+    return this
+        .<Void> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<EnableReplicationPeerRequest, EnableReplicationPeerResponse, Void> call(controller,
+                stub, RequestConverter.buildEnableReplicationPeerRequest(peerId),
+                (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> null)).call();
+  }
+
+  @Override
+  public CompletableFuture<Void> disableReplicationPeer(String peerId) {
+    return this
+        .<Void> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<DisableReplicationPeerRequest, DisableReplicationPeerResponse, Void> call(
+                controller, stub, RequestConverter.buildDisableReplicationPeerRequest(peerId), (s,
+                    c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> null))
+        .call();
+  }
+
+  public CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId) {
+    return this
+        .<ReplicationPeerConfig> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig> call(
+                controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId), (
+                    s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
+                (resp) -> ReplicationSerDeHelper.convert(resp.getPeerConfig()))).call();
+  }
+
+  @Override
+  public CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
+      ReplicationPeerConfig peerConfig) {
+    return this
+        .<Void> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<UpdateReplicationPeerConfigRequest, UpdateReplicationPeerConfigResponse, Void> call(
+                controller, stub, RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId,
+                  peerConfig), (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done), (
+                    resp) -> null)).call();
+  }
+
+  @Override
+  public CompletableFuture<Void> appendReplicationPeerTableCFs(String id,
+      Map<TableName, ? extends Collection<String>> tableCfs) {
+    if (tableCfs == null) {
+      return failedFuture(new ReplicationException("tableCfs is null"));
+    }
+
+    CompletableFuture<Void> future = new CompletableFuture<Void>();
+    getReplicationPeerConfig(id).whenComplete((peerConfig, error) -> {
+      if (!completeExceptionally(future, error)) {
+        ReplicationSerDeHelper.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
+        updateReplicationPeerConfig(id, peerConfig).whenComplete((result, err) -> {
+          if (!completeExceptionally(future, error)) {
+            future.complete(result);
+          }
+        });
+      }
+    });
+    return future;
+  }
+
+  @Override
+  public CompletableFuture<Void> removeReplicationPeerTableCFs(String id,
+      Map<TableName, ? extends Collection<String>> tableCfs) {
+    if (tableCfs == null) {
+      return failedFuture(new ReplicationException("tableCfs is null"));
+    }
+
+    CompletableFuture<Void> future = new CompletableFuture<Void>();
+    getReplicationPeerConfig(id).whenComplete((peerConfig, error) -> {
+      if (!completeExceptionally(future, error)) {
+        try {
+          ReplicationSerDeHelper.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
+        } catch (ReplicationException e) {
+          future.completeExceptionally(e);
+          return;
         }
+        updateReplicationPeerConfig(id, peerConfig).whenComplete((result, err) -> {
+          if (!completeExceptionally(future, error)) {
+            future.complete(result);
+          }
+        });
+      }
+    });
+    return future;
+  }
+
+  @Override
+  public CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers() {
+    return listReplicationPeers((Pattern) null);
+  }
 
-        @Override
-        public void onComplete() {
-          future.complete(settings);
+  @Override
+  public CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(String regex) {
+    return listReplicationPeers(Pattern.compile(regex));
+  }
+
+  @Override
+  public CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern) {
+    return this
+        .<List<ReplicationPeerDescription>> newMasterCaller()
+        .action(
+          (controller, stub) -> this
+              .<ListReplicationPeersRequest, ListReplicationPeersResponse, List<ReplicationPeerDescription>> call(
+                controller,
+                stub,
+                RequestConverter.buildListReplicationPeersRequest(pattern),
+                (s, c, req, done) -> s.listReplicationPeers(c, req, done),
+                (resp) -> resp.getPeerDescList().stream()
+                    .map(ReplicationSerDeHelper::toReplicationPeerDescription)
+                    .collect(Collectors.toList()))).call();
+  }
+
+  @Override
+  public CompletableFuture<List<TableCFs>> listReplicatedTableCFs() {
+    CompletableFuture<List<TableCFs>> future = new CompletableFuture<List<TableCFs>>();
+    listTables().whenComplete(
+      (tables, error) -> {
+        if (!completeExceptionally(future, error)) {
+          List<TableCFs> replicatedTableCFs = new ArrayList<>();
+          Arrays.asList(tables).forEach(
+            table -> {
+              Map<String, Integer> cfs = new HashMap<>();
+              Arrays.asList(table.getColumnFamilies()).stream()
+                  .filter(column -> column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL)
+                  .forEach(column -> {
+                    cfs.put(column.getNameAsString(), column.getScope());
+                  });
+              if (!cfs.isEmpty()) {
+                replicatedTableCFs.add(new TableCFs(table.getTableName(), cfs));
+              }
+            });
+          future.complete(replicatedTableCFs);
         }
       });
     return future;
@@ -1470,4 +1661,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
     future.completeExceptionally(error);
     return future;
   }
+
+  private <T> boolean completeExceptionally(CompletableFuture<T> future, Throwable error) {
+    if (error != null) {
+      future.completeExceptionally(error);
+      return true;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40cc666a/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 cadd6cc..8063070 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
@@ -3869,31 +3869,7 @@ public class HBaseAdmin implements Admin {
       throw new ReplicationException("tableCfs is null");
     }
     ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
-    Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
-    if (preTableCfs == null) {
-      peerConfig.setTableCFsMap(tableCfs);
-    } else {
-      for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-        TableName table = entry.getKey();
-        Collection<String> appendCfs = entry.getValue();
-        if (preTableCfs.containsKey(table)) {
-          List<String> cfs = preTableCfs.get(table);
-          if (cfs == null || appendCfs == null || appendCfs.isEmpty()) {
-            preTableCfs.put(table, null);
-          } else {
-            Set<String> cfSet = new HashSet<String>(cfs);
-            cfSet.addAll(appendCfs);
-            preTableCfs.put(table, Lists.newArrayList(cfSet));
-          }
-        } else {
-          if (appendCfs == null || appendCfs.isEmpty()) {
-            preTableCfs.put(table, null);
-          } else {
-            preTableCfs.put(table, Lists.newArrayList(appendCfs));
-          }
-        }
-      }
-    }
+    ReplicationSerDeHelper.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig);
     updateReplicationPeerConfig(id, peerConfig);
   }
 
@@ -3905,37 +3881,7 @@ public class HBaseAdmin implements Admin {
       throw new ReplicationException("tableCfs is null");
     }
     ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
-    Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
-    if (preTableCfs == null) {
-      throw new ReplicationException("Table-Cfs for peer" + id + " is null");
-    }
-    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-
-      TableName table = entry.getKey();
-      Collection<String> removeCfs = entry.getValue();
-      if (preTableCfs.containsKey(table)) {
-        List<String> cfs = preTableCfs.get(table);
-        if (cfs == null && (removeCfs == null || removeCfs.isEmpty())) {
-          preTableCfs.remove(table);
-        } else if (cfs != null && (removeCfs != null && !removeCfs.isEmpty())) {
-          Set<String> cfSet = new HashSet<String>(cfs);
-          cfSet.removeAll(removeCfs);
-          if (cfSet.isEmpty()) {
-            preTableCfs.remove(table);
-          } else {
-            preTableCfs.put(table, Lists.newArrayList(cfSet));
-          }
-        } else if (cfs == null && (removeCfs != null && !removeCfs.isEmpty())) {
-          throw new ReplicationException("Cannot remove cf of table: " + table
-              + " which doesn't specify cfs from table-cfs config in peer: " + id);
-        } else if (cfs != null && (removeCfs == null || removeCfs.isEmpty())) {
-          throw new ReplicationException("Cannot remove table: " + table
-              + " which has specified cfs from table-cfs config in peer: " + id);
-        }
-      } else {
-        throw new ReplicationException("No table: " + table + " in table-cfs config of peer: " + id);
-      }
-    }
+    ReplicationSerDeHelper.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id);
     updateReplicationPeerConfig(id, peerConfig);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/40cc666a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
index 2d5539c..f561f4a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
@@ -30,11 +30,14 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.util.Collection;
 import java.util.HashSet;
@@ -367,4 +370,68 @@ public final class ReplicationSerDeHelper {
     builder.setConfig(convert(desc.getPeerConfig()));
     return builder.build();
   }
+
+  public static void appendTableCFsToReplicationPeerConfig(
+      Map<TableName, ? extends Collection<String>> tableCfs, ReplicationPeerConfig peerConfig) {
+    Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
+    if (preTableCfs == null) {
+      peerConfig.setTableCFsMap(tableCfs);
+    } else {
+      for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
+        TableName table = entry.getKey();
+        Collection<String> appendCfs = entry.getValue();
+        if (preTableCfs.containsKey(table)) {
+          List<String> cfs = preTableCfs.get(table);
+          if (cfs == null || appendCfs == null || appendCfs.isEmpty()) {
+            preTableCfs.put(table, null);
+          } else {
+            Set<String> cfSet = new HashSet<String>(cfs);
+            cfSet.addAll(appendCfs);
+            preTableCfs.put(table, Lists.newArrayList(cfSet));
+          }
+        } else {
+          if (appendCfs == null || appendCfs.isEmpty()) {
+            preTableCfs.put(table, null);
+          } else {
+            preTableCfs.put(table, Lists.newArrayList(appendCfs));
+          }
+        }
+      }
+    }
+  }
+
+  public static void removeTableCFsFromReplicationPeerConfig(
+      Map<TableName, ? extends Collection<String>> tableCfs, ReplicationPeerConfig peerConfig,
+      String id) throws ReplicationException {
+    Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
+    if (preTableCfs == null) {
+      throw new ReplicationException("Table-Cfs for peer: " + id + " is null");
+    }
+    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
+      TableName table = entry.getKey();
+      Collection<String> removeCfs = entry.getValue();
+      if (preTableCfs.containsKey(table)) {
+        List<String> cfs = preTableCfs.get(table);
+        if (cfs == null && (removeCfs == null || removeCfs.isEmpty())) {
+          preTableCfs.remove(table);
+        } else if (cfs != null && (removeCfs != null && !removeCfs.isEmpty())) {
+          Set<String> cfSet = new HashSet<String>(cfs);
+          cfSet.removeAll(removeCfs);
+          if (cfSet.isEmpty()) {
+            preTableCfs.remove(table);
+          } else {
+            preTableCfs.put(table, Lists.newArrayList(cfSet));
+          }
+        } else if (cfs == null && (removeCfs != null && !removeCfs.isEmpty())) {
+          throw new ReplicationException("Cannot remove cf of table: " + table
+              + " which doesn't specify cfs from table-cfs config in peer: " + id);
+        } else if (cfs != null && (removeCfs == null || removeCfs.isEmpty())) {
+          throw new ReplicationException("Cannot remove table: " + table
+              + " which has specified cfs from table-cfs config in peer: " + id);
+        }
+      } else {
+        throw new ReplicationException("No table: " + table + " in table-cfs config of peer: " + id);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40cc666a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java
index f0dee0a..1881d4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java
@@ -34,7 +34,7 @@ import org.junit.BeforeClass;
  */
 public abstract class TestAsyncAdminBase {
 
-  protected static final Log LOG = LogFactory.getLog(TestAdmin1.class);
+  protected static final Log LOG = LogFactory.getLog(TestAsyncAdminBase.class);
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected static byte[] FAMILY = Bytes.toBytes("testFamily");
   protected static final byte[] FAMILY_0 = Bytes.toBytes("cf0");

http://git-wip-us.apache.org/repos/asf/hbase/blob/40cc666a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
new file mode 100644
index 0000000..c850c38
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
@@ -0,0 +1,416 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletionException;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Class to test asynchronous replication admin operations.
+ */
+@Category({MediumTests.class, ClientTests.class})
+public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
+
+  private final String ID_ONE = "1";
+  private final String KEY_ONE = "127.0.0.1:2181:/hbase";
+  private final String ID_SECOND = "2";
+  private final String KEY_SECOND = "127.0.0.1:2181:/hbase2";
+
+  @Rule
+  public TestName name = new TestName();
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
+    TEST_UTIL.startMiniCluster();
+    ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+  }
+
+  @Test
+  public void testAddRemovePeer() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+    rpc2.setClusterKey(KEY_SECOND);
+    // Add a valid peer
+    admin.addReplicationPeer(ID_ONE, rpc1).join();
+    // try adding the same (fails)
+    try {
+      admin.addReplicationPeer(ID_ONE, rpc1).join();
+      fail("Test case should fail as adding a same peer.");
+    } catch (CompletionException e) {
+      // OK!
+    }
+    assertEquals(1, admin.listReplicationPeers().get().size());
+    // Try to remove an inexisting peer
+    try {
+      admin.removeReplicationPeer(ID_SECOND).join();
+      fail("Test case should fail as removing a inexisting peer.");
+    } catch (CompletionException e) {
+      // OK!
+    }
+    assertEquals(1, admin.listReplicationPeers().get().size());
+    // Add a second since multi-slave is supported
+    admin.addReplicationPeer(ID_SECOND, rpc2).join();
+    assertEquals(2, admin.listReplicationPeers().get().size());
+    // Remove the first peer we added
+    admin.removeReplicationPeer(ID_ONE).join();
+    assertEquals(1, admin.listReplicationPeers().get().size());
+    admin.removeReplicationPeer(ID_SECOND).join();
+    assertEquals(0, admin.listReplicationPeers().get().size());
+  }
+
+  @Test
+  public void testPeerConfig() throws Exception {
+    ReplicationPeerConfig config = new ReplicationPeerConfig();
+    config.setClusterKey(KEY_ONE);
+    config.getConfiguration().put("key1", "value1");
+    config.getConfiguration().put("key2", "value2");
+    admin.addReplicationPeer(ID_ONE, config).join();
+
+    List<ReplicationPeerDescription> peers = admin.listReplicationPeers().get();
+    assertEquals(1, peers.size());
+    ReplicationPeerDescription peerOne = peers.get(0);
+    assertNotNull(peerOne);
+    assertEquals("value1", peerOne.getPeerConfig().getConfiguration().get("key1"));
+    assertEquals("value2", peerOne.getPeerConfig().getConfiguration().get("key2"));
+
+    admin.removeReplicationPeer(ID_ONE).join();
+  }
+
+  @Test
+  public void testEnableDisablePeer() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    admin.addReplicationPeer(ID_ONE, rpc1).join();
+    List<ReplicationPeerDescription> peers = admin.listReplicationPeers().get();
+    assertEquals(1, peers.size());
+    assertTrue(peers.get(0).isEnabled());
+
+    admin.disableReplicationPeer(ID_ONE).join();
+    peers = admin.listReplicationPeers().get();
+    assertEquals(1, peers.size());
+    assertFalse(peers.get(0).isEnabled());
+    admin.removeReplicationPeer(ID_ONE).join();
+  }
+
+  @Test
+  public void testAppendPeerTableCFs() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    final TableName tableName1 = TableName.valueOf(name.getMethodName() + "t1");
+    final TableName tableName2 = TableName.valueOf(name.getMethodName() + "t2");
+    final TableName tableName3 = TableName.valueOf(name.getMethodName() + "t3");
+    final TableName tableName4 = TableName.valueOf(name.getMethodName() + "t4");
+    final TableName tableName5 = TableName.valueOf(name.getMethodName() + "t5");
+    final TableName tableName6 = TableName.valueOf(name.getMethodName() + "t6");
+
+    // Add a valid peer
+    admin.addReplicationPeer(ID_ONE, rpc1).join();
+
+    Map<TableName, List<String>> tableCFs = new HashMap<>();
+
+    // append table t1 to replication
+    tableCFs.put(tableName1, null);
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    Map<TableName, List<String>> result = admin.getReplicationPeerConfig(ID_ONE).get()
+        .getTableCFsMap();
+    assertEquals(1, result.size());
+    assertEquals(true, result.containsKey(tableName1));
+    assertNull(result.get(tableName1));
+
+    // append table t2 to replication
+    tableCFs.clear();
+    tableCFs.put(tableName2, null);
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    result = admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap();
+    assertEquals(2, result.size());
+    assertTrue("Should contain t1", result.containsKey(tableName1));
+    assertTrue("Should contain t2", result.containsKey(tableName2));
+    assertNull(result.get(tableName1));
+    assertNull(result.get(tableName2));
+
+    // append table column family: f1 of t3 to replication
+    tableCFs.clear();
+    tableCFs.put(tableName3, new ArrayList<>());
+    tableCFs.get(tableName3).add("f1");
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    result = admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap();
+    assertEquals(3, result.size());
+    assertTrue("Should contain t1", result.containsKey(tableName1));
+    assertTrue("Should contain t2", result.containsKey(tableName2));
+    assertTrue("Should contain t3", result.containsKey(tableName3));
+    assertNull(result.get(tableName1));
+    assertNull(result.get(tableName2));
+    assertEquals(1, result.get(tableName3).size());
+    assertEquals("f1", result.get(tableName3).get(0));
+
+    // append table column family: f1,f2 of t4 to replication
+    tableCFs.clear();
+    tableCFs.put(tableName4, new ArrayList<>());
+    tableCFs.get(tableName4).add("f1");
+    tableCFs.get(tableName4).add("f2");
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    result = admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap();
+    assertEquals(4, result.size());
+    assertTrue("Should contain t1", result.containsKey(tableName1));
+    assertTrue("Should contain t2", result.containsKey(tableName2));
+    assertTrue("Should contain t3", result.containsKey(tableName3));
+    assertTrue("Should contain t4", result.containsKey(tableName4));
+    assertNull(result.get(tableName1));
+    assertNull(result.get(tableName2));
+    assertEquals(1, result.get(tableName3).size());
+    assertEquals("f1", result.get(tableName3).get(0));
+    assertEquals(2, result.get(tableName4).size());
+    assertEquals("f1", result.get(tableName4).get(0));
+    assertEquals("f2", result.get(tableName4).get(1));
+
+    // append "table5" => [], then append "table5" => ["f1"]
+    tableCFs.clear();
+    tableCFs.put(tableName5, new ArrayList<>());
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    tableCFs.clear();
+    tableCFs.put(tableName5, new ArrayList<>());
+    tableCFs.get(tableName5).add("f1");
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    result = admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap();
+    assertEquals(5, result.size());
+    assertTrue("Should contain t5", result.containsKey(tableName5));
+    // null means replication all cfs of tab5
+    assertNull(result.get(tableName5));
+
+    // append "table6" => ["f1"], then append "table6" => []
+    tableCFs.clear();
+    tableCFs.put(tableName6, new ArrayList<>());
+    tableCFs.get(tableName6).add("f1");
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    tableCFs.clear();
+    tableCFs.put(tableName6, new ArrayList<>());
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    result = admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap();
+    assertEquals(6, result.size());
+    assertTrue("Should contain t6", result.containsKey(tableName6));
+    // null means replication all cfs of tab6
+    assertNull(result.get(tableName6));
+
+    admin.removeReplicationPeer(ID_ONE).join();
+  }
+
+  @Test
+  public void testRemovePeerTableCFs() throws Exception {
+    ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
+    rpc1.setClusterKey(KEY_ONE);
+    final TableName tableName1 = TableName.valueOf(name.getMethodName() + "t1");
+    final TableName tableName2 = TableName.valueOf(name.getMethodName() + "t2");
+    final TableName tableName3 = TableName.valueOf(name.getMethodName() + "t3");
+    final TableName tableName4 = TableName.valueOf(name.getMethodName() + "t4");
+    // Add a valid peer
+    admin.addReplicationPeer(ID_ONE, rpc1).join();
+    Map<TableName, List<String>> tableCFs = new HashMap<>();
+    try {
+      tableCFs.put(tableName3, null);
+      admin.removeReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+      fail("Test case should fail as removing table-cfs from a peer whose table-cfs is null");
+    } catch (CompletionException e) {
+      assertTrue(e.getCause() instanceof ReplicationException);
+    }
+    assertNull(admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap());
+
+    tableCFs.clear();
+    tableCFs.put(tableName1, null);
+    tableCFs.put(tableName2, new ArrayList<>());
+    tableCFs.get(tableName2).add("cf1");
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    try {
+      tableCFs.clear();
+      tableCFs.put(tableName3, null);
+      admin.removeReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+      fail("Test case should fail as removing table-cfs from a peer whose table-cfs didn't contain t3");
+    } catch (CompletionException e) {
+      assertTrue(e.getCause() instanceof ReplicationException);
+    }
+    Map<TableName, List<String>> result = admin.getReplicationPeerConfig(ID_ONE).get()
+        .getTableCFsMap();
+    assertEquals(2, result.size());
+    assertTrue("Should contain t1", result.containsKey(tableName1));
+    assertTrue("Should contain t2", result.containsKey(tableName2));
+    assertNull(result.get(tableName1));
+    assertEquals(1, result.get(tableName2).size());
+    assertEquals("cf1", result.get(tableName2).get(0));
+
+    try {
+      tableCFs.clear();
+      tableCFs.put(tableName1, new ArrayList<>());
+      tableCFs.get(tableName1).add("cf1");
+      admin.removeReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+      fail("Test case should fail, because table t1 didn't specify cfs in peer config");
+    } catch (CompletionException e) {
+      assertTrue(e.getCause() instanceof ReplicationException);
+    }
+    tableCFs.clear();
+    tableCFs.put(tableName1, null);
+    admin.removeReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    result = admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap();
+    assertEquals(1, result.size());
+    assertEquals(1, result.get(tableName2).size());
+    assertEquals("cf1", result.get(tableName2).get(0));
+
+    try {
+      tableCFs.clear();
+      tableCFs.put(tableName2, null);
+      admin.removeReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+      fail("Test case should fail, because table t2 hase specified cfs in peer config");
+    } catch (CompletionException e) {
+      assertTrue(e.getCause() instanceof ReplicationException);
+    }
+    tableCFs.clear();
+    tableCFs.put(tableName2, new ArrayList<>());
+    tableCFs.get(tableName2).add("cf1");
+    admin.removeReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    assertNull(admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap());
+
+    tableCFs.clear();
+    tableCFs.put(tableName4, new ArrayList<>());
+    admin.appendReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    admin.removeReplicationPeerTableCFs(ID_ONE, tableCFs).join();
+    assertNull(admin.getReplicationPeerConfig(ID_ONE).get().getTableCFsMap());
+
+    admin.removeReplicationPeer(ID_ONE);
+  }
+
+  @Test
+  public void testSetPeerNamespaces() throws Exception {
+    String ns1 = "ns1";
+    String ns2 = "ns2";
+
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(KEY_ONE);
+    admin.addReplicationPeer(ID_ONE, rpc).join();
+
+    // add ns1 and ns2 to peer config
+    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    Set<String> namespaces = new HashSet<>();
+    namespaces.add(ns1);
+    namespaces.add(ns2);
+    rpc.setNamespaces(namespaces);
+    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    namespaces = admin.getReplicationPeerConfig(ID_ONE).get().getNamespaces();
+    assertEquals(2, namespaces.size());
+    assertTrue(namespaces.contains(ns1));
+    assertTrue(namespaces.contains(ns2));
+
+    // update peer config only contains ns1
+    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    namespaces.clear();
+    namespaces.add(ns1);
+    rpc.setNamespaces(namespaces);
+    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    namespaces = admin.getReplicationPeerConfig(ID_ONE).get().getNamespaces();
+    assertEquals(1, namespaces.size());
+    assertTrue(namespaces.contains(ns1));
+
+    admin.removeReplicationPeer(ID_ONE).join();
+  }
+
+  @Test
+  public void testNamespacesAndTableCfsConfigConflict() throws Exception {
+    String ns1 = "ns1";
+    String ns2 = "ns2";
+    final TableName tableName1 = TableName.valueOf(ns1 + ":" + name.getMethodName());
+    final TableName tableName2 = TableName.valueOf(ns2 + ":" + name.getMethodName() + "2");
+
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(KEY_ONE);
+    admin.addReplicationPeer(ID_ONE, rpc).join();
+
+    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    Set<String> namespaces = new HashSet<String>();
+    namespaces.add(ns1);
+    rpc.setNamespaces(namespaces);
+    admin.updateReplicationPeerConfig(ID_ONE, rpc).get();
+    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    Map<TableName, List<String>> tableCfs = new HashMap<>();
+    tableCfs.put(tableName1, new ArrayList<>());
+    rpc.setTableCFsMap(tableCfs);
+    try {
+      admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+      fail("Test case should fail, because table " + tableName1 + " conflict with namespace " + ns1);
+    } catch (CompletionException e) {
+      // OK
+    }
+
+    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    tableCfs.clear();
+    tableCfs.put(tableName2, new ArrayList<>());
+    rpc.setTableCFsMap(tableCfs);
+    admin.updateReplicationPeerConfig(ID_ONE, rpc).get();
+    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    namespaces.clear();
+    namespaces.add(ns2);
+    rpc.setNamespaces(namespaces);
+    try {
+      admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+      fail("Test case should fail, because namespace " + ns2 + " conflict with table " + tableName2);
+    } catch (CompletionException e) {
+      // OK
+    }
+
+    admin.removeReplicationPeer(ID_ONE).join();
+  }
+
+  @Test
+  public void testPeerBandwidth() throws Exception {
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(KEY_ONE);
+
+    admin.addReplicationPeer(ID_ONE, rpc).join();
+    rpc = admin.getReplicationPeerConfig(ID_ONE).get();
+    assertEquals(0, rpc.getBandwidth());
+
+    rpc.setBandwidth(2097152);
+    admin.updateReplicationPeerConfig(ID_ONE, rpc).join();
+    assertEquals(2097152, admin.getReplicationPeerConfig(ID_ONE).join().getBandwidth());
+
+    admin.removeReplicationPeer(ID_ONE).join();
+  }
+}


[18/40] hbase git commit: HBASE-17937 Memstore size becomes negative in case of expensive postPut/Delete Coprocessor call

Posted by sy...@apache.org.
HBASE-17937 Memstore size becomes negative in case of expensive postPut/Delete Coprocessor call

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


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

Branch: refs/heads/hbase-12439
Commit: 49cba2c237ecc1b3285d942f1ad176ea50c44cd1
Parents: 40cc666
Author: Abhishek Singh Chouhan <ab...@gmail.com>
Authored: Wed Apr 19 11:22:23 2017 +0530
Committer: zhangduo <zh...@apache.org>
Committed: Fri Apr 21 08:50:09 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |   4 +-
 ...NegativeMemstoreSizeWithSlowCoprocessor.java | 104 +++++++++++++++++++
 2 files changed, 107 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/49cba2c2/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 b21a84d..4836dc8 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
@@ -3392,6 +3392,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         applyFamilyMapToMemstore(familyMaps[i], memstoreSize);
       }
 
+      // update memstore size
+      this.addAndGetMemstoreSize(memstoreSize);
+
       // calling the post CP hook for batch mutation
       if (!replay && coprocessorHost != null) {
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
@@ -3444,7 +3447,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     } finally {
       // Call complete rather than completeAndWait because we probably had error if walKey != null
       if (writeEntry != null) mvcc.complete(writeEntry);
-      this.addAndGetMemstoreSize(memstoreSize);
       if (locked) {
         this.updatesLock.readLock().unlock();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/49cba2c2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java
new file mode 100644
index 0000000..ae2f055
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java
@@ -0,0 +1,104 @@
+/**
+ * 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.coprocessor;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+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.Table;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemstoreSize;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test that verifies we do not have memstore size negative when a postPut/Delete hook is
+ * slow/expensive and a flush is triggered at the same time the coprocessow is doing its work. To
+ * simulate this we call flush from the coprocessor itself
+ */
+@Category(LargeTests.class)
+public class TestNegativeMemstoreSizeWithSlowCoprocessor {
+
+  static final Log LOG = LogFactory.getLog(TestNegativeMemstoreSizeWithSlowCoprocessor.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] tableName = Bytes.toBytes("test_table");
+  private static final byte[] family = Bytes.toBytes("f");
+  private static final byte[] qualifier = Bytes.toBytes("q");
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
+      FlushingRegionObserver.class.getName());
+    conf.setBoolean(CoprocessorHost.ABORT_ON_ERROR_KEY, true);
+    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); // Let's fail fast.
+    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.createTable(TableName.valueOf(tableName), family);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testNegativeMemstoreSize() throws IOException, InterruptedException {
+    boolean IOEthrown = false;
+    Table table = null;
+    try {
+      table = TEST_UTIL.getConnection().getTable(TableName.valueOf(tableName));
+
+      // Adding data
+      Put put1 = new Put(Bytes.toBytes("row1"));
+      put1.addColumn(family, qualifier, Bytes.toBytes("Value1"));
+      table.put(put1);
+      Put put2 = new Put(Bytes.toBytes("row2"));
+      put2.addColumn(family, qualifier, Bytes.toBytes("Value2"));
+      table.put(put2);
+      table.put(put2);
+    } catch (IOException e) {
+      IOEthrown = true;
+    } finally {
+      Assert.assertFalse("Shouldn't have thrown an exception", IOEthrown);
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  public static class FlushingRegionObserver extends SimpleRegionObserver {
+
+    @Override
+    public void postPut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
+        final WALEdit edit, final Durability durability) throws IOException {
+      HRegion region = (HRegion) c.getEnvironment().getRegion();
+      super.postPut(c, put, edit, durability);
+
+      if (Bytes.equals(put.getRow(), Bytes.toBytes("row2"))) {
+        region.flush(false);
+        Assert.assertTrue(region.addAndGetMemstoreSize(new MemstoreSize()) >= 0);
+      }
+    }
+  }
+}
\ No newline at end of file


[16/40] hbase git commit: HBASE-17542 Move backup system table into separate namespace

Posted by sy...@apache.org.
HBASE-17542 Move backup system table into separate namespace


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

Branch: refs/heads/hbase-12439
Commit: b1ef8dd43aa0f0102f296ea9b3eb76b5623052f5
Parents: 0953c14
Author: tedyu <yu...@gmail.com>
Authored: Thu Apr 20 02:57:24 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Apr 20 02:57:24 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/BackupHFileCleaner.java  |  8 ++++----
 .../hbase/backup/BackupRestoreConstants.java     |  2 +-
 .../hbase/backup/impl/BackupSystemTable.java     | 19 +++++++++++++++++++
 3 files changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b1ef8dd4/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
index b6b4c0a..8e6e843 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
@@ -54,9 +54,9 @@ public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abor
   private boolean aborted;
   private Configuration conf;
   private Connection connection;
-  private long prevReadFromBackupTbl = 0, // timestamp of most recent read from hbase:backup table
-      secondPrevReadFromBackupTbl = 0; // timestamp of 2nd most recent read from hbase:backup table
-  //used by unit test to skip reading hbase:backup
+  private long prevReadFromBackupTbl = 0, // timestamp of most recent read from backup:system table
+      secondPrevReadFromBackupTbl = 0; // timestamp of 2nd most recent read from backup:system table
+  //used by unit test to skip reading backup:system
   private boolean checkForFullyBackedUpTables = true;
   private List<TableName> fullyBackedUpTables = null;
 
@@ -117,7 +117,7 @@ public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abor
     Iterable<FileStatus> deletables = Iterables.filter(files, new Predicate<FileStatus>() {
       @Override
       public boolean apply(FileStatus file) {
-        // If the file is recent, be conservative and wait for one more scan of hbase:backup table
+        // If the file is recent, be conservative and wait for one more scan of backup:system table
         if (file.getModificationTime() > secondPrevReadFromBackupTbl) {
           return false;
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b1ef8dd4/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
index 770ccce..e46904b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -30,7 +30,7 @@ public interface BackupRestoreConstants {
    * Backup/Restore constants
    */
   public final static String BACKUP_SYSTEM_TABLE_NAME_KEY = "hbase.backup.system.table.name";
-  public final static String BACKUP_SYSTEM_TABLE_NAME_DEFAULT = "hbase:backup";
+  public final static String BACKUP_SYSTEM_TABLE_NAME_DEFAULT = "backup:system";
 
   public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl";
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b1ef8dd4/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
index 1ba8087..217e750 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo;
@@ -165,6 +166,8 @@ public final class BackupSystemTable implements Closeable {
   private void checkSystemTable() throws IOException {
     try (Admin admin = connection.getAdmin();) {
 
+      verifyNamespaceExists(admin);
+
       if (!admin.tableExists(tableName)) {
         HTableDescriptor backupHTD =
             BackupSystemTable.getSystemTableDescriptor(connection.getConfiguration());
@@ -174,6 +177,22 @@ public final class BackupSystemTable implements Closeable {
     }
   }
 
+  private void verifyNamespaceExists(Admin admin) throws IOException {
+      String namespaceName  = tableName.getNamespaceAsString();
+      NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build();
+      NamespaceDescriptor[] list = admin.listNamespaceDescriptors();
+      boolean exists = false;
+      for( NamespaceDescriptor nsd: list) {
+        if (nsd.getName().equals(ns.getName())) {
+          exists = true;
+          break;
+        }
+      }
+      if (!exists) {
+        admin.createNamespace(ns);
+      }
+  }
+
   private void waitForSystemTable(Admin admin) throws IOException {
     long TIMEOUT = 60000;
     long startTime = EnvironmentEdgeManager.currentTime();


[09/40] hbase git commit: HBASE-17914 Create a new reader instead of cloning a new StoreFile when compaction

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index d72529a..0ba500a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@ -17,11 +17,12 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import com.google.common.io.Closeables;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -59,8 +60,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
-import com.google.common.io.Closeables;
-
 /**
  * A compactor is a compaction algorithm associated a given policy. Base class also contains
  * reusable parts for implementing compactors (what is common and what isn't is evolving).
@@ -216,15 +215,9 @@ public abstract class Compactor<T extends CellSink> {
    * @param filesToCompact Files.
    * @return Scanners.
    */
-  protected List<StoreFileScanner> createFileScanners(
-      final Collection<StoreFile> filesToCompact,
-      long smallestReadPoint,
-      boolean useDropBehind) throws IOException {
-    return StoreFileScanner.getScannersForStoreFiles(filesToCompact,
-        /* cache blocks = */ false,
-        /* use pread = */ false,
-        /* is compaction */ true,
-        /* use Drop Behind */ useDropBehind,
+  protected List<StoreFileScanner> createFileScanners(Collection<StoreFile> filesToCompact,
+      long smallestReadPoint, boolean useDropBehind) throws IOException {
+    return StoreFileScanner.getScannersForCompaction(filesToCompact, useDropBehind,
       smallestReadPoint);
   }
 
@@ -281,8 +274,6 @@ public abstract class Compactor<T extends CellSink> {
     // Find the smallest read point across all the Scanners.
     long smallestReadPoint = getSmallestReadPoint();
 
-    List<StoreFileScanner> scanners;
-    Collection<StoreFile> readersToClose;
     T writer = null;
     boolean dropCache;
     if (request.isMajor() || request.isAllFiles()) {
@@ -291,22 +282,8 @@ public abstract class Compactor<T extends CellSink> {
       dropCache = this.dropCacheMinor;
     }
 
-    if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", true)) {
-      // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles,
-      // HFiles, and their readers
-      readersToClose = new ArrayList<>(request.getFiles().size());
-      for (StoreFile f : request.getFiles()) {
-        StoreFile clonedStoreFile = f.cloneForReader();
-        // create the reader after the store file is cloned in case
-        // the sequence id is used for sorting in scanners
-        clonedStoreFile.createReader();
-        readersToClose.add(clonedStoreFile);
-      }
-      scanners = createFileScanners(readersToClose, smallestReadPoint, dropCache);
-    } else {
-      readersToClose = Collections.emptyList();
-      scanners = createFileScanners(request.getFiles(), smallestReadPoint, dropCache);
-    }
+    List<StoreFileScanner> scanners =
+        createFileScanners(request.getFiles(), smallestReadPoint, dropCache);
     InternalScanner scanner = null;
     boolean finished = false;
     try {
@@ -336,13 +313,6 @@ public abstract class Compactor<T extends CellSink> {
       }
     } finally {
       Closeables.close(scanner, true);
-      for (StoreFile f : readersToClose) {
-        try {
-          f.closeReader(true);
-        } catch (IOException e) {
-          LOG.warn("Exception closing " + f, e);
-        }
-      }
       if (!finished && writer != null) {
         abortWriter(writer);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
index ace45ec..7b745ba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
@@ -133,7 +133,7 @@ public class CompressionTest {
     writer.appendFileInfo(Bytes.toBytes("compressioninfokey"), Bytes.toBytes("compressioninfoval"));
     writer.close();
     Cell cc = null;
-    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), true, conf);
     try {
       reader.loadFileInfo();
       HFileScanner scanner = reader.getScanner(false, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 4eab62b..dca02e4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -849,8 +849,8 @@ public class HBaseFsck extends Configured implements Closeable {
             FileStatus[] storeFiles = fs.listStatus(file.getPath());
             // For all the stores in this column family.
             for (FileStatus storeFile : storeFiles) {
-              HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(), new CacheConfig(
-                  getConf()), getConf());
+              HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(),
+                new CacheConfig(getConf()), true, getConf());
               if ((reader.getFirstKey() != null)
                   && ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
                       ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey()).getKey()) > 0))) {
@@ -954,7 +954,7 @@ public class HBaseFsck extends Configured implements Closeable {
         HFile.Reader hf = null;
         try {
           CacheConfig cacheConf = new CacheConfig(getConf());
-          hf = HFile.createReader(fs, hfile.getPath(), cacheConf, getConf());
+          hf = HFile.createReader(fs, hfile.getPath(), cacheConf, true, getConf());
           hf.loadFileInfo();
           Cell startKv = hf.getFirstKey();
           start = CellUtil.cloneRow(startKv);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
index 82200bd..e46e43b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
@@ -98,7 +98,7 @@ public class HFileCorruptionChecker {
   protected void checkHFile(Path p) throws IOException {
     HFile.Reader r = null;
     try {
-      r = HFile.createReader(fs, p, cacheConf, conf);
+      r = HFile.createReader(fs, p, cacheConf, true, conf);
     } catch (CorruptHFileException che) {
       LOG.warn("Found corrupt HFile " + p, che);
       corrupted.add(p);
@@ -230,7 +230,7 @@ public class HFileCorruptionChecker {
   protected void checkMobFile(Path p) throws IOException {
     HFile.Reader r = null;
     try {
-      r = HFile.createReader(fs, p, cacheConf, conf);
+      r = HFile.createReader(fs, p, cacheConf, true, conf);
     } catch (CorruptHFileException che) {
       LOG.warn("Found corrupt mob file " + p, che);
       corruptedMobFiles.add(p);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
index 562630a..37ca56b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
@@ -403,7 +403,7 @@ public class HFilePerformanceEvaluation {
 
     @Override
     void setUp() throws Exception {
-      reader = HFile.createReader(this.fs, this.mf, new CacheConfig(this.conf), this.conf);
+      reader = HFile.createReader(this.fs, this.mf, new CacheConfig(this.conf), true, this.conf);
       this.reader.loadFileInfo();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index b1a0d3c..7668aa2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -5174,6 +5174,7 @@ public class TestFromClientSide {
       assertEquals(2, store.getStorefilesCount());
       store.triggerMajorCompaction();
       region.compact(true);
+      store.closeAndArchiveCompactedFiles();
       waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max
       assertEquals(1, store.getStorefilesCount());
       expectedBlockCount -= 2; // evicted two blocks, cached none

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
index 6a0921f..0fd3cdb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,7 +50,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-@Category({IOTests.class, SmallTests.class})
+@Category({ IOTests.class, SmallTests.class })
 public class TestHalfStoreFileReader {
   private static HBaseTestingUtility TEST_UTIL;
 
@@ -64,19 +65,14 @@ public class TestHalfStoreFileReader {
   }
 
   /**
-   * Test the scanner and reseek of a half hfile scanner. The scanner API
-   * demands that seekTo and reseekTo() only return < 0 if the key lies
-   * before the start of the file (with no position on the scanner). Returning
-   * 0 if perfect match (rare), and return > 1 if we got an imperfect match.
-   *
-   * The latter case being the most common, we should generally be returning 1,
-   * and if we do, there may or may not be a 'next' in the scanner/file.
-   *
-   * A bug in the half file scanner was returning -1 at the end of the bottom
-   * half, and that was causing the infrastructure above to go null causing NPEs
-   * and other problems.  This test reproduces that failure, and also tests
-   * both the bottom and top of the file while we are at it.
-   *
+   * Test the scanner and reseek of a half hfile scanner. The scanner API demands that seekTo and
+   * reseekTo() only return < 0 if the key lies before the start of the file (with no position on
+   * the scanner). Returning 0 if perfect match (rare), and return > 1 if we got an imperfect match.
+   * The latter case being the most common, we should generally be returning 1, and if we do, there
+   * may or may not be a 'next' in the scanner/file. A bug in the half file scanner was returning -1
+   * at the end of the bottom half, and that was causing the infrastructure above to go null causing
+   * NPEs and other problems. This test reproduces that failure, and also tests both the bottom and
+   * top of the file while we are at it.
    * @throws IOException
    */
   @Test
@@ -88,10 +84,8 @@ public class TestHalfStoreFileReader {
     FileSystem fs = FileSystem.get(conf);
     CacheConfig cacheConf = new CacheConfig(conf);
     HFileContext meta = new HFileContextBuilder().withBlockSize(1024).build();
-    HFile.Writer w = HFile.getWriterFactory(conf, cacheConf)
-        .withPath(fs, p)
-        .withFileContext(meta)
-        .create();
+    HFile.Writer w =
+        HFile.getWriterFactory(conf, cacheConf).withPath(fs, p).withFileContext(meta).create();
 
     // write some things.
     List<KeyValue> items = genSomeKeys();
@@ -100,12 +94,12 @@ public class TestHalfStoreFileReader {
     }
     w.close();
 
-    HFile.Reader r = HFile.createReader(fs, p, cacheConf, conf);
+    HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf);
     r.loadFileInfo();
     Cell midKV = r.midkey();
     byte[] midkey = CellUtil.cloneRow(midKV);
 
-    //System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
+    // System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
 
     Reference bottom = new Reference(midkey, Reference.Range.bottom);
     doTestOfScanAndReseek(p, fs, bottom, cacheConf);
@@ -116,11 +110,10 @@ public class TestHalfStoreFileReader {
     r.close();
   }
 
-  private void doTestOfScanAndReseek(Path p, FileSystem fs, Reference bottom,
-      CacheConfig cacheConf)
+  private void doTestOfScanAndReseek(Path p, FileSystem fs, Reference bottom, CacheConfig cacheConf)
       throws IOException {
-    final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p,
-      cacheConf, bottom, TEST_UTIL.getConfiguration());
+    final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p, cacheConf, bottom, true,
+        new AtomicInteger(0), true, TEST_UTIL.getConfiguration());
     halfreader.loadFileInfo();
     final HFileScanner scanner = halfreader.getScanner(false, false);
 
@@ -128,110 +121,103 @@ public class TestHalfStoreFileReader {
     Cell curr;
     do {
       curr = scanner.getCell();
-      KeyValue reseekKv =
-          getLastOnCol(curr);
+      KeyValue reseekKv = getLastOnCol(curr);
       int ret = scanner.reseekTo(reseekKv);
       assertTrue("reseek to returned: " + ret, ret > 0);
-      //System.out.println(curr + ": " + ret);
+      // System.out.println(curr + ": " + ret);
     } while (scanner.next());
 
     int ret = scanner.reseekTo(getLastOnCol(curr));
-    //System.out.println("Last reseek: " + ret);
-    assertTrue( ret > 0 );
+    // System.out.println("Last reseek: " + ret);
+    assertTrue(ret > 0);
 
     halfreader.close(true);
   }
 
-
   // Tests the scanner on an HFile that is backed by HalfStoreFiles
   @Test
   public void testHalfScanner() throws IOException {
-      String root_dir = TEST_UTIL.getDataTestDir().toString();
-      Path p = new Path(root_dir, "test");
-      Configuration conf = TEST_UTIL.getConfiguration();
-      FileSystem fs = FileSystem.get(conf);
-      CacheConfig cacheConf = new CacheConfig(conf);
-      HFileContext meta = new HFileContextBuilder().withBlockSize(1024).build();
-      HFile.Writer w = HFile.getWriterFactory(conf, cacheConf)
-              .withPath(fs, p)
-              .withFileContext(meta)
-              .create();
-
-      // write some things.
-      List<KeyValue> items = genSomeKeys();
-      for (KeyValue kv : items) {
-          w.append(kv);
-      }
-      w.close();
+    String root_dir = TEST_UTIL.getDataTestDir().toString();
+    Path p = new Path(root_dir, "test");
+    Configuration conf = TEST_UTIL.getConfiguration();
+    FileSystem fs = FileSystem.get(conf);
+    CacheConfig cacheConf = new CacheConfig(conf);
+    HFileContext meta = new HFileContextBuilder().withBlockSize(1024).build();
+    HFile.Writer w =
+        HFile.getWriterFactory(conf, cacheConf).withPath(fs, p).withFileContext(meta).create();
 
+    // write some things.
+    List<KeyValue> items = genSomeKeys();
+    for (KeyValue kv : items) {
+      w.append(kv);
+    }
+    w.close();
 
-      HFile.Reader r = HFile.createReader(fs, p, cacheConf, conf);
-      r.loadFileInfo();
-      Cell midKV = r.midkey();
-      byte[] midkey = CellUtil.cloneRow(midKV);
+    HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf);
+    r.loadFileInfo();
+    Cell midKV = r.midkey();
+    byte[] midkey = CellUtil.cloneRow(midKV);
 
-      Reference bottom = new Reference(midkey, Reference.Range.bottom);
-      Reference top = new Reference(midkey, Reference.Range.top);
+    Reference bottom = new Reference(midkey, Reference.Range.bottom);
+    Reference top = new Reference(midkey, Reference.Range.top);
 
-      // Ugly code to get the item before the midkey
-      KeyValue beforeMidKey = null;
-      for (KeyValue item : items) {
-          if (CellComparator.COMPARATOR.compare(item, midKV) >= 0) {
-              break;
-          }
-          beforeMidKey = item;
+    // Ugly code to get the item before the midkey
+    KeyValue beforeMidKey = null;
+    for (KeyValue item : items) {
+      if (CellComparator.COMPARATOR.compare(item, midKV) >= 0) {
+        break;
       }
-      System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
-      System.out.println("beforeMidKey: " + beforeMidKey);
-
+      beforeMidKey = item;
+    }
+    System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
+    System.out.println("beforeMidKey: " + beforeMidKey);
 
-      // Seek on the splitKey, should be in top, not in bottom
-      Cell foundKeyValue = doTestOfSeekBefore(p, fs, bottom, midKV, cacheConf);
-      assertEquals(beforeMidKey, foundKeyValue);
+    // Seek on the splitKey, should be in top, not in bottom
+    Cell foundKeyValue = doTestOfSeekBefore(p, fs, bottom, midKV, cacheConf);
+    assertEquals(beforeMidKey, foundKeyValue);
 
-      // Seek tot the last thing should be the penultimate on the top, the one before the midkey on the bottom.
-      foundKeyValue = doTestOfSeekBefore(p, fs, top, items.get(items.size() - 1), cacheConf);
-      assertEquals(items.get(items.size() - 2), foundKeyValue);
+    // Seek tot the last thing should be the penultimate on the top, the one before the midkey on
+    // the bottom.
+    foundKeyValue = doTestOfSeekBefore(p, fs, top, items.get(items.size() - 1), cacheConf);
+    assertEquals(items.get(items.size() - 2), foundKeyValue);
 
-      foundKeyValue = doTestOfSeekBefore(p, fs, bottom, items.get(items.size() - 1), cacheConf);
-      assertEquals(beforeMidKey, foundKeyValue);
+    foundKeyValue = doTestOfSeekBefore(p, fs, bottom, items.get(items.size() - 1), cacheConf);
+    assertEquals(beforeMidKey, foundKeyValue);
 
-      // Try and seek before something that is in the bottom.
-      foundKeyValue = doTestOfSeekBefore(p, fs, top, items.get(0), cacheConf);
-      assertNull(foundKeyValue);
+    // Try and seek before something that is in the bottom.
+    foundKeyValue = doTestOfSeekBefore(p, fs, top, items.get(0), cacheConf);
+    assertNull(foundKeyValue);
 
-      // Try and seek before the first thing.
-      foundKeyValue = doTestOfSeekBefore(p, fs, bottom, items.get(0), cacheConf);
-      assertNull(foundKeyValue);
+    // Try and seek before the first thing.
+    foundKeyValue = doTestOfSeekBefore(p, fs, bottom, items.get(0), cacheConf);
+    assertNull(foundKeyValue);
 
-      // Try and seek before the second thing in the top and bottom.
-      foundKeyValue = doTestOfSeekBefore(p, fs, top, items.get(1), cacheConf);
-      assertNull(foundKeyValue);
+    // Try and seek before the second thing in the top and bottom.
+    foundKeyValue = doTestOfSeekBefore(p, fs, top, items.get(1), cacheConf);
+    assertNull(foundKeyValue);
 
-      foundKeyValue = doTestOfSeekBefore(p, fs, bottom, items.get(1), cacheConf);
-      assertEquals(items.get(0), foundKeyValue);
+    foundKeyValue = doTestOfSeekBefore(p, fs, bottom, items.get(1), cacheConf);
+    assertEquals(items.get(0), foundKeyValue);
 
-      // Try to seek before the splitKey in the top file
-      foundKeyValue = doTestOfSeekBefore(p, fs, top, midKV, cacheConf);
-      assertNull(foundKeyValue);
-    }
+    // Try to seek before the splitKey in the top file
+    foundKeyValue = doTestOfSeekBefore(p, fs, top, midKV, cacheConf);
+    assertNull(foundKeyValue);
+  }
 
   private Cell doTestOfSeekBefore(Path p, FileSystem fs, Reference bottom, Cell seekBefore,
-                                        CacheConfig cacheConfig)
-            throws IOException {
-      final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p,
-              cacheConfig, bottom, TEST_UTIL.getConfiguration());
-      halfreader.loadFileInfo();
-      final HFileScanner scanner = halfreader.getScanner(false, false);
-      scanner.seekBefore(seekBefore);
-      return scanner.getCell();
+      CacheConfig cacheConfig) throws IOException {
+    final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p, cacheConfig, bottom, true,
+        new AtomicInteger(0), true, TEST_UTIL.getConfiguration());
+    halfreader.loadFileInfo();
+    final HFileScanner scanner = halfreader.getScanner(false, false);
+    scanner.seekBefore(seekBefore);
+    return scanner.getCell();
   }
 
   private KeyValue getLastOnCol(Cell curr) {
-    return KeyValueUtil.createLastOnRow(
-        curr.getRowArray(), curr.getRowOffset(), curr.getRowLength(),
-        curr.getFamilyArray(), curr.getFamilyOffset(), curr.getFamilyLength(),
-        curr.getQualifierArray(), curr.getQualifierOffset(), curr.getQualifierLength());
+    return KeyValueUtil.createLastOnRow(curr.getRowArray(), curr.getRowOffset(),
+      curr.getRowLength(), curr.getFamilyArray(), curr.getFamilyOffset(), curr.getFamilyLength(),
+      curr.getQualifierArray(), curr.getQualifierOffset(), curr.getQualifierLength());
   }
 
   static final int SIZE = 1000;
@@ -244,18 +230,10 @@ public class TestHalfStoreFileReader {
     List<KeyValue> ret = new ArrayList<>(SIZE);
     for (int i = 0; i < SIZE; i++) {
       KeyValue kv =
-          new KeyValue(
-              _b(String.format("row_%04d", i)),
-              _b("family"),
-              _b("qualifier"),
-              1000, // timestamp
+          new KeyValue(_b(String.format("row_%04d", i)), _b("family"), _b("qualifier"), 1000, // timestamp
               _b("value"));
       ret.add(kv);
     }
     return ret;
   }
-
-
-
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index 3315b6f..49807a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -248,7 +248,7 @@ public class TestCacheOnWrite {
   }
 
   private void readStoreFile(boolean useTags) throws IOException {
-    HFile.Reader reader = HFile.createReader(fs, storeFilePath, cacheConf, conf);
+    HFile.Reader reader = HFile.createReader(fs, storeFilePath, cacheConf, true, conf);
     LOG.info("HFile information: " + reader);
     HFileContext meta = new HFileContextBuilder().withCompression(compress)
       .withBytesPerCheckSum(CKBYTES).withChecksumType(ChecksumType.NULL)

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/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 4db459a..d209430 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
@@ -161,7 +161,7 @@ public class TestHFile  {
     Writer w =
         HFile.getWriterFactory(conf, cacheConf).withPath(fs, f).withFileContext(context).create();
     w.close();
-    Reader r = HFile.createReader(fs, f, cacheConf, conf);
+    Reader r = HFile.createReader(fs, f, cacheConf, true, conf);
     r.loadFileInfo();
     assertNull(r.getFirstKey());
     assertNull(r.getLastKey());
@@ -178,7 +178,7 @@ public class TestHFile  {
     fsos.close();
 
     try {
-      Reader r = HFile.createReader(fs, f, cacheConf, conf);
+      Reader r = HFile.createReader(fs, f, cacheConf, true, conf);
     } catch (CorruptHFileException che) {
       // Expected failure
       return;
@@ -218,7 +218,7 @@ public class TestHFile  {
     truncateFile(fs, w.getPath(), trunc);
 
     try {
-      Reader r = HFile.createReader(fs, trunc, cacheConf, conf);
+      Reader r = HFile.createReader(fs, trunc, cacheConf, true, conf);
     } catch (CorruptHFileException che) {
       // Expected failure
       return;
@@ -453,7 +453,7 @@ public class TestHFile  {
       writer.append(kv);
       writer.close();
       fout.close();
-      Reader reader = HFile.createReader(fs, mFile, cacheConf, conf);
+      Reader reader = HFile.createReader(fs, mFile, cacheConf, true, conf);
       reader.loadFileInfo();
       assertNull(reader.getMetaBlock("non-existant", false));
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index 28930db..2052c1d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -565,7 +565,7 @@ public class TestHFileBlockIndex {
    conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, false);
 
    // Read the HFile
-   HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, conf);
+   HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, true, conf);
 
    boolean hasArrayIndexOutOfBoundsException = false;
    try {
@@ -644,7 +644,7 @@ public class TestHFileBlockIndex {
       }
 
       // Read the HFile
-      HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, conf);
+      HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, true, conf);
       assertEquals(expectedNumLevels,
           reader.getTrailer().getNumDataIndexLevels());
 
@@ -774,7 +774,7 @@ public class TestHFileBlockIndex {
     }
     hfw.close();
 
-    HFile.Reader reader = HFile.createReader(fs, hfPath, cacheConf, conf);
+    HFile.Reader reader = HFile.createReader(fs, hfPath, cacheConf, true, conf);
     // Scanner doesn't do Cells yet.  Fix.
     HFileScanner scanner = reader.getScanner(true, true);
     for (int i = 0; i < keys.size(); ++i) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
index 3264558..40e9ab7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
@@ -178,7 +178,7 @@ public class TestHFileEncryption {
     }
 
     // read it back in and validate correct crypto metadata
-    HFile.Reader reader = HFile.createReader(fs, path, cacheConf, conf);
+    HFile.Reader reader = HFile.createReader(fs, path, cacheConf, true, conf);
     try {
       reader.loadFileInfo();
       FixedFileTrailer trailer = reader.getTrailer();
@@ -230,7 +230,7 @@ public class TestHFileEncryption {
         LOG.info("Reading with " + fileContext);
         int i = 0;
         HFileScanner scanner = null;
-        HFile.Reader reader = HFile.createReader(fs, path, cacheConf, conf);
+        HFile.Reader reader = HFile.createReader(fs, path, cacheConf, true, conf);
         try {
           reader.loadFileInfo();
           FixedFileTrailer trailer = reader.getTrailer();
@@ -252,7 +252,7 @@ public class TestHFileEncryption {
 
         // Test random seeks with pread
         LOG.info("Random seeking with " + fileContext);
-        reader = HFile.createReader(fs, path, cacheConf, conf);
+        reader = HFile.createReader(fs, path, cacheConf, true, conf);
         try {
           scanner = reader.getScanner(false, true);
           assertTrue("Initial seekTo failed", scanner.seekTo());

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
index f1528c2..686024d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
@@ -77,7 +77,7 @@ public class TestHFileInlineToRootChunkConversion {
     }
     hfw.close();
 
-    HFile.Reader reader = HFile.createReader(fs, hfPath, cacheConf, conf);
+    HFile.Reader reader = HFile.createReader(fs, hfPath, cacheConf, true, conf);
     // Scanner doesn't do Cells yet.  Fix.
     HFileScanner scanner = reader.getScanner(true, true);
     for (int i = 0; i < keys.size(); ++i) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
index 4c3db03..dfa5ee8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
@@ -81,7 +81,7 @@ public class TestPrefetch {
 
   private void readStoreFile(Path storeFilePath) throws Exception {
     // Open the file
-    HFile.Reader reader = HFile.createReader(fs, storeFilePath, cacheConf, conf);
+    HFile.Reader reader = HFile.createReader(fs, storeFilePath, cacheConf, true, conf);
 
     while (!reader.prefetchComplete()) {
       // Sleep for a bit

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
index a9ecf7b..b3cd8ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
@@ -109,8 +109,8 @@ public class TestReseekTo {
     writer.close();
     fout.close();
 
-    HFile.Reader reader = HFile.createReader(TEST_UTIL.getTestFileSystem(),
-        ncTFile, cacheConf, TEST_UTIL.getConfiguration());
+    HFile.Reader reader = HFile.createReader(TEST_UTIL.getTestFileSystem(), ncTFile, cacheConf,
+      true, TEST_UTIL.getConfiguration());
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, true);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
index d46af4a..f4309ea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
@@ -136,7 +136,7 @@ public class TestSeekBeforeWithInlineBlocks {
           }
   
           // Read the HFile
-          HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, conf);
+          HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, true, conf);
           
           // Sanity check the HFile index level
           assertEquals(expectedNumLevels, reader.getTrailer().getNumDataIndexLevels());

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
index 6531d2c..b268f0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
@@ -147,7 +147,7 @@ public class TestSeekTo {
     Path p = makeNewFile(tagUsage);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Configuration conf = TEST_UTIL.getConfiguration();
-    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, true);
     assertFalse(scanner.seekBefore(toKV("a", tagUsage)));
@@ -206,7 +206,7 @@ public class TestSeekTo {
     Path p = makeNewFile(tagUsage);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Configuration conf = TEST_UTIL.getConfiguration();
-    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, true);
     assertFalse(scanner.seekBefore(toKV("a", tagUsage)));
@@ -300,7 +300,7 @@ public class TestSeekTo {
     Path p = makeNewFile(tagUsage);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Configuration conf = TEST_UTIL.getConfiguration();
-    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     assertEquals(2, reader.getDataBlockIndexReader().getRootBlockCount());
     HFileScanner scanner = reader.getScanner(false, true);
@@ -338,7 +338,7 @@ public class TestSeekTo {
     Path p = makeNewFile(tagUsage);
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Configuration conf = TEST_UTIL.getConfiguration();
-    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     HFileBlockIndex.BlockIndexReader blockIndexReader =
       reader.getDataBlockIndexReader();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index 20fc992..274a76e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -365,8 +365,8 @@ public class TestHFileOutputFormat2  {
       FileStatus[] file = fs.listStatus(sub1[0].getPath());
 
       // open as HFile Reader and pull out TIMERANGE FileInfo.
-      HFile.Reader rd = HFile.createReader(fs, file[0].getPath(),
-          new CacheConfig(conf), conf);
+      HFile.Reader rd =
+          HFile.createReader(fs, file[0].getPath(), new CacheConfig(conf), true, conf);
       Map<byte[],byte[]> finfo = rd.loadFileInfo();
       byte[] range = finfo.get("TIMERANGE".getBytes());
       assertNotNull(range);
@@ -458,8 +458,8 @@ public class TestHFileOutputFormat2  {
       RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(dir, true);
       while(iterator.hasNext()) {
         LocatedFileStatus keyFileStatus = iterator.next();
-        HFile.Reader reader = HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf),
-            conf);
+        HFile.Reader reader =
+            HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf), true, conf);
         HFileScanner scanner = reader.getScanner(false, false, false);
         scanner.seekTo();
         Cell cell = scanner.getCell();
@@ -1043,7 +1043,7 @@ public class TestHFileOutputFormat2  {
         // verify that the compression on this file matches the configured
         // compression
         Path dataFilePath = fs.listStatus(f.getPath())[0].getPath();
-        Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), conf);
+        Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), true, conf);
         Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
 
         byte[] bloomFilter = fileInfo.get(StoreFile.BLOOM_FILTER_TYPE_KEY);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
index b8d973b..8967ac7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
@@ -480,7 +480,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
    */
   private static int getKVCountFromHfile(FileSystem fs, Path p) throws IOException {
     Configuration conf = util.getConfiguration();
-    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, false);
     scanner.seekTo();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
index b7d5c6f..efcf91e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
@@ -556,7 +556,7 @@ public class TestImportTsv implements Configurable {
    */
   private static int getKVCountFromHfile(FileSystem fs, Path p) throws IOException {
     Configuration conf = util.getConfiguration();
-    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, false);
     scanner.seekTo();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
index 7ae5afc..7f1723c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
@@ -620,8 +620,8 @@ public class TestLoadIncrementalHFiles {
 
   private int verifyHFile(Path p) throws IOException {
     Configuration conf = util.getConfiguration();
-    HFile.Reader reader = HFile.createReader(
-        p.getFileSystem(conf), p, new CacheConfig(conf), conf);
+    HFile.Reader reader =
+        HFile.createReader(p.getFileSystem(conf), p, new CacheConfig(conf), true, conf);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, false);
     scanner.seekTo();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
index 84a2ba7..6647ffe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java
@@ -60,8 +60,8 @@ public class TestMobFile extends TestCase {
     String caseName = getName();
     MobTestUtil.writeStoreFile(writer, caseName);
 
-    MobFile mobFile = new MobFile(new StoreFile(fs, writer.getPath(),
-        conf, cacheConf, BloomType.NONE));
+    MobFile mobFile =
+        new MobFile(new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
     byte[] family = Bytes.toBytes(caseName);
     byte[] qualify = Bytes.toBytes(caseName);
 
@@ -112,8 +112,8 @@ public class TestMobFile extends TestCase {
             .build();
     MobTestUtil.writeStoreFile(writer, getName());
 
-    MobFile mobFile = new MobFile(new StoreFile(fs, writer.getPath(),
-        conf, cacheConf, BloomType.NONE));
+    MobFile mobFile =
+        new MobFile(new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
     assertNotNull(mobFile.getScanner());
     assertTrue(mobFile.getScanner() instanceof StoreFileScanner);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
index 83936aa..47a1c24 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
@@ -772,9 +772,7 @@ public class TestMobCompactor {
     ResultScanner results = table.getScanner(scan);
     int count = 0;
     for (Result res : results) {
-      for (Cell cell : res.listCells()) {
-        count++;
-      }
+      count += res.size();
     }
     results.close();
     return count;
@@ -817,8 +815,9 @@ public class TestMobCompactor {
       Path path = files[0].getPath();
       CacheConfig cacheConf = new CacheConfig(conf);
       StoreFile sf = new StoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
-        BloomType.NONE);
-      HFile.Reader reader = sf.createReader().getHFileReader();
+        BloomType.NONE, true);
+      sf.initReader();
+      HFile.Reader reader = sf.getReader().getHFileReader();
       byte[] encryptionKey = reader.getTrailer().getEncryptionKey();
       Assert.assertTrue(null != encryptionKey);
       Assert.assertTrue(reader.getFileContext().getEncryptionContext().getCipher().getName()

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
index 290e6f4..f65e224 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
@@ -515,10 +515,11 @@ public class TestPartitionedMobCompactor {
       try {
         for (CompactionDelPartition delPartition : request.getDelPartitions()) {
           for (Path newDelPath : delPartition.listDelFiles()) {
-            StoreFile sf = new StoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE);
-            // pre-create reader of a del file to avoid race condition when opening the reader in each
-            // partition.
-            sf.createReader();
+            StoreFile sf =
+                new StoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true);
+            // pre-create reader of a del file to avoid race condition when opening the reader in
+            // each partition.
+            sf.initReader();
             delPartition.addStoreFile(sf);
           }
         }
@@ -768,7 +769,6 @@ public class TestPartitionedMobCompactor {
    * @param delPartitions all del partitions
    */
   private void compareDelFiles(List<CompactionDelPartition> delPartitions) {
-    int i = 0;
     Map<Path, Path> delMap = new HashMap<>();
     for (CompactionDelPartition delPartition : delPartitions) {
       for (Path f : delPartition.listDelFiles()) {
@@ -850,12 +850,12 @@ public class TestPartitionedMobCompactor {
   private int countDelCellsInDelFiles(List<Path> paths) throws IOException {
     List<StoreFile> sfs = new ArrayList<>();
     int size = 0;
-    for(Path path : paths) {
-      StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE);
+    for (Path path : paths) {
+      StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
       sfs.add(sf);
     }
-    List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
-        false, false, HConstants.LATEST_TIMESTAMP);
+    List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,
+      false, true, false, false, HConstants.LATEST_TIMESTAMP));
     Scan scan = new Scan();
     scan.setMaxVersions(hcd.getMaxVersions());
     long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
index a074a9a..e36d16f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
@@ -592,10 +592,9 @@ public class DataBlockEncodingTool {
     Path path = new Path(hfilePath);
     CacheConfig cacheConf = new CacheConfig(conf);
     FileSystem fs = FileSystem.get(conf);
-    StoreFile hsf = new StoreFile(fs, path, conf, cacheConf,
-      BloomType.NONE);
-
-    StoreFileReader reader = hsf.createReader();
+    StoreFile hsf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
+    hsf.initReader();
+    StoreFileReader reader = hsf.getReader();
     reader.loadFileInfo();
     KeyValueScanner scanner = reader.getStoreFileScanner(true, true, false, 0, 0, false);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
index eb77c28..f47fc4e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
@@ -60,9 +60,9 @@ public class EncodedSeekPerformanceTest {
 
     // read all of the key values
     StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(),
-        path, configuration, cacheConf, BloomType.NONE);
-
-    StoreFileReader reader = storeFile.createReader();
+        path, configuration, cacheConf, BloomType.NONE, true);
+    storeFile.initReader();
+    StoreFileReader reader = storeFile.getReader();
     StoreFileScanner scanner = reader.getStoreFileScanner(true, false, false, 0, 0, false);
     Cell current;
 
@@ -90,11 +90,11 @@ public class EncodedSeekPerformanceTest {
       List<Cell> seeks) throws IOException {
     // read all of the key values
     StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(),
-      path, configuration, cacheConf, BloomType.NONE);
-
+      path, configuration, cacheConf, BloomType.NONE, true);
+    storeFile.initReader();
     long totalSize = 0;
 
-    StoreFileReader reader = storeFile.createReader();
+    StoreFileReader reader = storeFile.getReader();
     StoreFileScanner scanner = reader.getStoreFileScanner(true, false, false, 0, 0, false);
 
     long startReadingTime = System.nanoTime();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
index 1169434..d52c6c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
@@ -48,7 +48,7 @@ public class MockStoreFile extends StoreFile {
   MockStoreFile(HBaseTestingUtility testUtil, Path testPath,
       long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException {
     super(testUtil.getTestFileSystem(), testPath, testUtil.getConfiguration(),
-      new CacheConfig(testUtil.getConfiguration()), BloomType.NONE);
+        new CacheConfig(testUtil.getConfiguration()), BloomType.NONE, true);
     this.length = length;
     this.isRef = isRef;
     this.ageInDisk = ageInDisk;
@@ -126,6 +126,11 @@ public class MockStoreFile extends StoreFile {
   }
 
   @Override
+  public boolean isCompactedAway() {
+    return compactedAway;
+  }
+
+  @Override
   public long getModificationTimeStamp() {
     return modificationTime;
   }
@@ -136,11 +141,22 @@ public class MockStoreFile extends StoreFile {
   }
 
   @Override
+  public void initReader() throws IOException {
+  }
+
+  @Override
+  public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
+      boolean pread, boolean isCompaction, long readPt, long scannerOrder,
+      boolean canOptimizeForNonNullColumn) throws IOException {
+    return getReader().getStoreFileScanner(cacheBlocks, pread, isCompaction, readPt, scannerOrder,
+      canOptimizeForNonNullColumn);
+  }
+
+  @Override
   public StoreFileReader getReader() {
     final long len = this.length;
     final TimeRangeTracker timeRangeTracker = this.timeRangeTracker;
     final long entries = this.entryCount;
-    final boolean compactedAway = this.compactedAway;
     return new StoreFileReader() {
       @Override
       public long length() {
@@ -158,11 +174,6 @@ public class MockStoreFile extends StoreFile {
       }
 
       @Override
-      public boolean isCompactedAway() {
-        return compactedAway;
-      }
-
-      @Override
       public void close(boolean evictOnClose) throws IOException {
         // no-op
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 9fed202..efe0605 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -218,9 +218,9 @@ public class TestCacheOnWriteInSchema {
   private void readStoreFile(Path path) throws IOException {
     CacheConfig cacheConf = store.getCacheConfig();
     BlockCache cache = cacheConf.getBlockCache();
-    StoreFile sf = new StoreFile(fs, path, conf, cacheConf,
-      BloomType.ROWCOL);
-    HFile.Reader reader = sf.createReader().getHFileReader();
+    StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
+    sf.initReader();
+    HFile.Reader reader = sf.getReader().getHFileReader();
     try {
       // Open a scanner with (on read) caching disabled
       HFileScanner scanner = reader.getScanner(false, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/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..58dbe8d 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
@@ -40,15 +40,12 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.experimental.categories.Category;
 
-@Category(SmallTests.class)
 public class TestCompactionPolicy {
   private final static Log LOG = LogFactory.getLog(TestCompactionPolicy.class);
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
index dfea761..57a5f59 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
@@ -200,8 +200,9 @@ public class TestCompoundBloomFilter {
 
   private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
       Path sfPath) throws IOException {
-    StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt);
-    StoreFileReader r = sf.createReader();
+    StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt, true);
+    sf.initReader();
+    StoreFileReader r = sf.getReader();
     final boolean pread = true; // does not really matter
     StoreFileScanner scanner = r.getStoreFileScanner(true, pread, false, 0, 0, false);
 
@@ -285,7 +286,7 @@ public class TestCompoundBloomFilter {
 
   private boolean isInBloom(StoreFileScanner scanner, byte[] row,
       byte[] qualifier) {
-    Scan scan = new Scan(row, row);
+    Scan scan = new Scan().withStartRow(row).withStopRow(row, true);
     scan.addColumn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME), qualifier);
     Store store = mock(Store.class);
     HColumnDescriptor hcd = mock(HColumnDescriptor.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
index b34c307..3e7477d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
@@ -280,7 +280,7 @@ public class TestEncryptionKeyRotation {
 
   private static byte[] extractHFileKey(Path path) throws Exception {
     HFile.Reader reader = HFile.createReader(TEST_UTIL.getTestFileSystem(), path,
-      new CacheConfig(conf), conf);
+      new CacheConfig(conf), true, conf);
     try {
       reader.loadFileInfo();
       Encryption.Context cryptoContext = reader.getFileContext().getEncryptionContext();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
index 2b0ab7b..3d8eeed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
@@ -66,7 +66,7 @@ public class TestEncryptionRandomKeying {
 
   private static byte[] extractHFileKey(Path path) throws Exception {
     HFile.Reader reader = HFile.createReader(TEST_UTIL.getTestFileSystem(), path,
-      new CacheConfig(conf), conf);
+      new CacheConfig(conf), true, conf);
     try {
       reader.loadFileInfo();
       Encryption.Context cryptoContext = reader.getFileContext().getEncryptionContext();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
index 9f0975d..e231b60 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
@@ -94,10 +94,10 @@ public class TestFSErrorsExposed {
     TestStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 
-    StoreFile sf = new StoreFile(fs, writer.getPath(),
-      util.getConfiguration(), cacheConf, BloomType.NONE);
-
-    StoreFileReader reader = sf.createReader();
+    StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
+        BloomType.NONE, true);
+    sf.initReader();
+    StoreFileReader reader = sf.getReader();
     HFileScanner scanner = reader.getScanner(false, true);
 
     FaultyInputStream inStream = faultyfs.inStreams.get(0).get();
@@ -144,8 +144,8 @@ public class TestFSErrorsExposed {
     TestStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 
-    StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(),
-      cacheConf, BloomType.NONE);
+    StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
+        BloomType.NONE, true);
 
     List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
         Collections.singletonList(sf), false, true, false, false,

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
index a50dc42..1997b31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
@@ -293,8 +293,9 @@ public class TestMobStoreCompaction {
     if (fs.exists(mobDirPath)) {
       FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath);
       for (FileStatus file : files) {
-        StoreFile sf = new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE);
-        Map<byte[], byte[]> fileInfo = sf.createReader().loadFileInfo();
+        StoreFile sf = new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true);
+        sf.initReader();
+        Map<byte[], byte[]> fileInfo = sf.getReader().loadFileInfo();
         byte[] count = fileInfo.get(StoreFile.MOB_CELLS_COUNT);
         assertTrue(count != null);
         mobCellsCount += Bytes.toLong(count);
@@ -407,7 +408,7 @@ public class TestMobStoreCompaction {
     int size = 0;
     if (fs.exists(mobDirPath)) {
       for (FileStatus f : fs.listStatus(mobDirPath)) {
-        StoreFile sf = new StoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE);
+        StoreFile sf = new StoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
         sfs.add(sf);
         if (StoreFileInfo.isDelFile(sf.getPath())) {
           numDelfiles++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
index ecb808e..bf0fb05 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
@@ -108,8 +108,8 @@ public class TestReversibleScanners {
           .withFileContext(hFileContext).build();
       writeStoreFile(writer);
 
-      StoreFile sf = new StoreFile(fs, writer.getPath(),
-          TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE);
+      StoreFile sf = new StoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+          BloomType.NONE, true);
 
       List<StoreFileScanner> scanners = StoreFileScanner
           .getScannersForStoreFiles(Collections.singletonList(sf),
@@ -162,11 +162,11 @@ public class TestReversibleScanners {
     writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
         writer2 });
 
-    StoreFile sf1 = new StoreFile(fs, writer1.getPath(),
-        TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE);
+    StoreFile sf1 = new StoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+        BloomType.NONE, true);
 
-    StoreFile sf2 = new StoreFile(fs, writer2.getPath(),
-        TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE);
+    StoreFile sf2 = new StoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+        BloomType.NONE, true);
     /**
      * Test without MVCC
      */
@@ -252,11 +252,11 @@ public class TestReversibleScanners {
     writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
         writer2 });
 
-    StoreFile sf1 = new StoreFile(fs, writer1.getPath(),
-        TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE);
+    StoreFile sf1 = new StoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+        BloomType.NONE, true);
 
-    StoreFile sf2 = new StoreFile(fs, writer2.getPath(),
-        TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE);
+    StoreFile sf2 = new StoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+        BloomType.NONE, true);
 
     ScanType scanType = ScanType.USER_SCAN;
     ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), FAMILYNAME, 0, Integer.MAX_VALUE,
@@ -272,7 +272,7 @@ public class TestReversibleScanners {
     // Case 2.Test reversed scan with a specified start row
     int startRowNum = ROWSIZE / 2;
     byte[] startRow = ROWS[startRowNum];
-    scan.setStartRow(startRow);
+    scan.withStartRow(startRow);
     storeScanner = getReversibleStoreScanner(memstore, sf1, sf2, scan,
         scanType, scanInfo, MAXMVCC);
     verifyCountAndOrder(storeScanner, QUALSIZE * (startRowNum + 1),
@@ -354,21 +354,21 @@ public class TestReversibleScanners {
 
     // Case5: Case4 + specify start row
     int startRowNum = ROWSIZE * 3 / 4;
-    scan.setStartRow(ROWS[startRowNum]);
+    scan.withStartRow(ROWS[startRowNum]);
     scanner = region.getScanner(scan);
     verifyCountAndOrder(scanner, (startRowNum + 1) * 2 * 2, (startRowNum + 1),
         false);
 
     // Case6: Case4 + specify stop row
     int stopRowNum = ROWSIZE / 4;
-    scan.setStartRow(HConstants.EMPTY_BYTE_ARRAY);
-    scan.setStopRow(ROWS[stopRowNum]);
+    scan.withStartRow(HConstants.EMPTY_BYTE_ARRAY);
+    scan.withStopRow(ROWS[stopRowNum]);
     scanner = region.getScanner(scan);
     verifyCountAndOrder(scanner, (ROWSIZE - stopRowNum - 1) * 2 * 2, (ROWSIZE
         - stopRowNum - 1), false);
 
     // Case7: Case4 + specify start row + specify stop row
-    scan.setStartRow(ROWS[startRowNum]);
+    scan.withStartRow(ROWS[startRowNum]);
     scanner = region.getScanner(scan);
     verifyCountAndOrder(scanner, (startRowNum - stopRowNum) * 2 * 2,
         (startRowNum - stopRowNum), false);
@@ -595,9 +595,6 @@ public class TestReversibleScanners {
 
       // Case2: seek to the previous row in backwardSeek
     int seekRowNum = ROWSIZE - 3;
-    KeyValue seekKey = KeyValueUtil.createLastOnRow(ROWS[seekRowNum]);
-      expectedKey = getNextReadableKeyValueWithBackwardScan(seekRowNum - 1, 0,
-          readPoint);
     res = false;
     for (KeyValueScanner scanner : scanners) {
       res |= scanner.backwardSeek(expectedKey);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/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..76bf1cc 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
@@ -276,7 +276,7 @@ public class TestStore {
     writer.close();
 
     // Verify that compression and encoding settings are respected
-    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), conf);
+    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), true, conf);
     Assert.assertEquals(hcd.getCompressionType(), reader.getCompressionAlgorithm());
     Assert.assertEquals(hcd.getDataBlockEncoding(), reader.getDataBlockEncoding());
     reader.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
index 7e4ebd8..d1444c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
@@ -27,6 +27,7 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -116,8 +117,7 @@ public class TestStoreFile extends HBaseTestCase {
     writeStoreFile(writer);
 
     Path sfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
-    StoreFile sf = new StoreFile(this.fs, sfPath, conf, cacheConf,
-      BloomType.NONE);
+    StoreFile sf = new StoreFile(this.fs, sfPath, conf, cacheConf, BloomType.NONE, true);
     checkHalfHFile(regionFs, sf);
   }
 
@@ -169,9 +169,9 @@ public class TestStoreFile extends HBaseTestCase {
     writeStoreFile(writer);
 
     Path hsfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
-    StoreFile hsf = new StoreFile(this.fs, hsfPath, conf, cacheConf,
-      BloomType.NONE);
-    StoreFileReader reader = hsf.createReader();
+    StoreFile hsf = new StoreFile(this.fs, hsfPath, conf, cacheConf, BloomType.NONE, true);
+    hsf.initReader();
+    StoreFileReader reader = hsf.getReader();
     // Split on a row, not in middle of row.  Midkey returned by reader
     // may be in middle of row.  Create new one with empty column and
     // timestamp.
@@ -184,11 +184,11 @@ public class TestStoreFile extends HBaseTestCase {
     // Make a reference
     HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow);
     Path refPath = splitStoreFile(regionFs, splitHri, TEST_FAMILY, hsf, midRow, true);
-    StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
-      BloomType.NONE);
+    StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf, BloomType.NONE, true);
+    refHsf.initReader();
     // Now confirm that I can read from the reference and that it only gets
     // keys from top half of the file.
-    HFileScanner s = refHsf.createReader().getScanner(false, false);
+    HFileScanner s = refHsf.getReader().getScanner(false, false);
     for(boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) {
       ByteBuffer bb = ByteBuffer.wrap(((KeyValue) s.getKey()).getKey());
       kv = KeyValueUtil.createKeyValueFromKey(bb);
@@ -242,13 +242,14 @@ public class TestStoreFile extends HBaseTestCase {
 
     // Try to open store file from link
     StoreFileInfo storeFileInfo = new StoreFileInfo(testConf, this.fs, linkFilePath);
-    StoreFile hsf = new StoreFile(this.fs, storeFileInfo, testConf, cacheConf,
-      BloomType.NONE);
+    StoreFile hsf =
+        new StoreFile(this.fs, storeFileInfo, testConf, cacheConf, BloomType.NONE, true);
     assertTrue(storeFileInfo.isLink());
+    hsf.initReader();
 
     // Now confirm that I can read from the link
     int count = 1;
-    HFileScanner s = hsf.createReader().getScanner(false, false);
+    HFileScanner s = hsf.getReader().getScanner(false, false);
     s.seekTo();
     while (s.next()) {
       count++;
@@ -295,8 +296,8 @@ public class TestStoreFile extends HBaseTestCase {
     // <root>/clone/splitB/<cf>/<reftohfilelink>
     HRegionInfo splitHriA = new HRegionInfo(hri.getTable(), null, SPLITKEY);
     HRegionInfo splitHriB = new HRegionInfo(hri.getTable(), SPLITKEY, null);
-    StoreFile f = new StoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE);
-    f.createReader();
+    StoreFile f = new StoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE, true);
+    f.initReader();
     Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top
     Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom
     f.closeReader(true);
@@ -307,12 +308,12 @@ public class TestStoreFile extends HBaseTestCase {
     // reference to a hfile link.  This code in StoreFile that handles this case.
 
     // Try to open store file from link
-    StoreFile hsfA = new StoreFile(this.fs, pathA, testConf, cacheConf,
-      BloomType.NONE);
+    StoreFile hsfA = new StoreFile(this.fs, pathA, testConf, cacheConf, BloomType.NONE, true);
+    hsfA.initReader();
 
     // Now confirm that I can read from the ref to link
     int count = 1;
-    HFileScanner s = hsfA.createReader().getScanner(false, false);
+    HFileScanner s = hsfA.getReader().getScanner(false, false);
     s.seekTo();
     while (s.next()) {
       count++;
@@ -320,11 +321,11 @@ public class TestStoreFile extends HBaseTestCase {
     assertTrue(count > 0); // read some rows here
 
     // Try to open store file from link
-    StoreFile hsfB = new StoreFile(this.fs, pathB, testConf, cacheConf,
-      BloomType.NONE);
+    StoreFile hsfB = new StoreFile(this.fs, pathB, testConf, cacheConf, BloomType.NONE, true);
+    hsfB.initReader();
 
     // Now confirm that I can read from the ref to link
-    HFileScanner sB = hsfB.createReader().getScanner(false, false);
+    HFileScanner sB = hsfB.getReader().getScanner(false, false);
     sB.seekTo();
 
     //count++ as seekTo() will advance the scanner
@@ -339,7 +340,8 @@ public class TestStoreFile extends HBaseTestCase {
 
   private void checkHalfHFile(final HRegionFileSystem regionFs, final StoreFile f)
       throws IOException {
-    Cell midkey = f.createReader().midkey();
+    f.initReader();
+    Cell midkey = f.getReader().midkey();
     KeyValue midKV = (KeyValue)midkey;
     byte [] midRow = CellUtil.cloneRow(midKV);
     // Create top split.
@@ -351,10 +353,12 @@ public class TestStoreFile extends HBaseTestCase {
         midRow, null);
     Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false);
     // Make readers on top and bottom.
-    StoreFileReader top = new StoreFile(
-      this.fs, topPath, conf, cacheConf, BloomType.NONE).createReader();
-    StoreFileReader bottom = new StoreFile(
-      this.fs, bottomPath, conf, cacheConf, BloomType.NONE).createReader();
+    StoreFile topF = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
+    topF.initReader();
+    StoreFileReader top = topF.getReader();
+    StoreFile bottomF = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
+    bottomF.initReader();
+    StoreFileReader bottom = bottomF.getReader();
     ByteBuffer previous = null;
     LOG.info("Midkey: " + midKV.toString());
     ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midKV.getKey());
@@ -412,7 +416,9 @@ public class TestStoreFile extends HBaseTestCase {
 
       assertNull(bottomPath);
 
-      top = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE).createReader();
+      topF = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
+      topF.initReader();
+      top = topF.getReader();
       // Now read from the top.
       first = true;
       topScanner = top.getScanner(false, false);
@@ -449,8 +455,10 @@ public class TestStoreFile extends HBaseTestCase {
       topPath = splitStoreFile(regionFs,topHri, TEST_FAMILY, f, badmidkey, true);
       bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, badmidkey, false);
       assertNull(topPath);
-      bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
-        BloomType.NONE).createReader();
+
+      bottomF = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
+      bottomF.initReader();
+      bottom = bottomF.getReader();
       first = true;
       bottomScanner = bottom.getScanner(false, false);
       while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
@@ -502,7 +510,8 @@ public class TestStoreFile extends HBaseTestCase {
     }
     writer.close();
 
-    StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf);
+    StoreFileReader reader =
+        new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
     reader.loadFileInfo();
     reader.loadBloomfilter();
     StoreFileScanner scanner = getStoreFileScanner(reader, false, false);
@@ -590,7 +599,8 @@ public class TestStoreFile extends HBaseTestCase {
     }
     writer.close();
 
-    StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf);
+    StoreFileReader reader =
+        new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
     reader.loadFileInfo();
     reader.loadBloomfilter();
 
@@ -635,7 +645,8 @@ public class TestStoreFile extends HBaseTestCase {
     writeStoreFile(writer);
     writer.close();
 
-    StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf);
+    StoreFileReader reader =
+        new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
 
     // Now do reseek with empty KV to position to the beginning of the file
 
@@ -695,7 +706,8 @@ public class TestStoreFile extends HBaseTestCase {
       }
       writer.close();
 
-      StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf);
+      StoreFileReader reader =
+          new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
       reader.loadFileInfo();
       reader.loadBloomfilter();
       StoreFileScanner scanner = getStoreFileScanner(reader, false, false);
@@ -844,12 +856,13 @@ public class TestStoreFile extends HBaseTestCase {
     writer.close();
 
     StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
-      BloomType.NONE);
+      BloomType.NONE, true);
     Store store = mock(Store.class);
     HColumnDescriptor hcd = mock(HColumnDescriptor.class);
     when(hcd.getName()).thenReturn(family);
     when(store.getFamily()).thenReturn(hcd);
-    StoreFileReader reader = hsf.createReader();
+    hsf.initReader();
+    StoreFileReader reader = hsf.getReader();
     StoreFileScanner scanner = getStoreFileScanner(reader, false, false);
     TreeSet<byte[]> columns = new TreeSet<>(Bytes.BYTES_COMPARATOR);
     columns.add(qualifier);
@@ -901,11 +914,12 @@ public class TestStoreFile extends HBaseTestCase {
     Path pathCowOff = new Path(baseDir, "123456789");
     StoreFileWriter writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
     StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
-      BloomType.NONE);
+      BloomType.NONE, true);
     LOG.debug(hsf.getPath().toString());
 
     // Read this file, we should see 3 misses
-    StoreFileReader reader = hsf.createReader();
+    hsf.initReader();
+    StoreFileReader reader = hsf.getReader();
     reader.loadFileInfo();
     StoreFileScanner scanner = getStoreFileScanner(reader, true, true);
     scanner.seek(KeyValue.LOWESTKEY);
@@ -923,10 +937,11 @@ public class TestStoreFile extends HBaseTestCase {
     Path pathCowOn = new Path(baseDir, "123456788");
     writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
     hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
-      BloomType.NONE);
+      BloomType.NONE, true);
 
     // Read this file, we should see 3 hits
-    reader = hsf.createReader();
+    hsf.initReader();
+    reader = hsf.getReader();
     scanner = getStoreFileScanner(reader, true, true);
     scanner.seek(KeyValue.LOWESTKEY);
     while (scanner.next() != null);
@@ -938,15 +953,15 @@ public class TestStoreFile extends HBaseTestCase {
     reader.close(cacheConf.shouldEvictOnClose());
 
     // Let's read back the two files to ensure the blocks exactly match
-    hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
-      BloomType.NONE);
-    StoreFileReader readerOne = hsf.createReader();
+    hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
+    hsf.initReader();
+    StoreFileReader readerOne = hsf.getReader();
     readerOne.loadFileInfo();
     StoreFileScanner scannerOne = getStoreFileScanner(readerOne, true, true);
     scannerOne.seek(KeyValue.LOWESTKEY);
-    hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
-      BloomType.NONE);
-    StoreFileReader readerTwo = hsf.createReader();
+    hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
+    hsf.initReader();
+    StoreFileReader readerTwo = hsf.getReader();
     readerTwo.loadFileInfo();
     StoreFileScanner scannerTwo = getStoreFileScanner(readerTwo, true, true);
     scannerTwo.seek(KeyValue.LOWESTKEY);
@@ -977,9 +992,9 @@ public class TestStoreFile extends HBaseTestCase {
     // Let's close the first file with evict on close turned on
     conf.setBoolean("hbase.rs.evictblocksonclose", true);
     cacheConf = new CacheConfig(conf);
-    hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
-      BloomType.NONE);
-    reader = hsf.createReader();
+    hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
+    hsf.initReader();
+    reader = hsf.getReader();
     reader.close(cacheConf.shouldEvictOnClose());
 
     // We should have 3 new evictions but the evict count stat should not change. Eviction because
@@ -991,9 +1006,9 @@ public class TestStoreFile extends HBaseTestCase {
     // Let's close the second file with evict on close turned off
     conf.setBoolean("hbase.rs.evictblocksonclose", false);
     cacheConf = new CacheConfig(conf);
-    hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
-      BloomType.NONE);
-    reader = hsf.createReader();
+    hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
+    hsf.initReader();
+    reader = hsf.getReader();
     reader.close(cacheConf.shouldEvictOnClose());
 
     // We expect no changes
@@ -1078,9 +1093,10 @@ public class TestStoreFile extends HBaseTestCase {
             .build();
     writer.close();
 
-    StoreFile storeFile = new StoreFile(fs, writer.getPath(), conf,
-      cacheConf, BloomType.NONE);
-    StoreFileReader reader = storeFile.createReader();
+    StoreFile storeFile =
+        new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true);
+    storeFile.initReader();
+    StoreFileReader reader = storeFile.getReader();
 
     Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
     byte[] value = fileInfo.get(HFileDataBlockEncoder.DATA_BLOCK_ENCODING);

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
index d628dc8..3d3c79c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
@@ -23,23 +23,24 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagUtil;
-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.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+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;
@@ -74,7 +75,8 @@ public class TestStoreFileScannerWithTagCompression {
     writeStoreFile(writer);
     writer.close();
 
-    StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf);
+    StoreFileReader reader =
+        new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
     StoreFileScanner s = reader.getStoreFileScanner(false, false, false, 0, 0, false);
     try {
       // Now do reseek with empty KV to position to the beginning of the file

http://git-wip-us.apache.org/repos/asf/hbase/blob/66b616d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
index dff6919..170fba2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
@@ -67,9 +67,6 @@ public class TestCompactor {
     when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyLong(),
       anyBoolean())).thenReturn(mock(StoreFileScanner.class));
     when(sf.getReader()).thenReturn(r);
-    when(sf.createReader()).thenReturn(r);
-    when(sf.createReader(anyBoolean())).thenReturn(r);
-    when(sf.cloneForReader()).thenReturn(sf);
     when(sf.getMaxSequenceId()).thenReturn(maxSequenceId);
     return sf;
   }


[23/40] hbase git commit: HBASE-17946 Shell command compact_rs don't work (Guangxu Cheng)

Posted by sy...@apache.org.
HBASE-17946 Shell command compact_rs don't work (Guangxu Cheng)


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

Branch: refs/heads/hbase-12439
Commit: 68e48c456dc018775df792507087bf275bf3304f
Parents: d39f40e
Author: tedyu <yu...@gmail.com>
Authored: Fri Apr 21 06:54:44 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Apr 21 06:54:44 2017 -0700

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/shell/commands/compact_rs.rb | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/68e48c45/hbase-shell/src/main/ruby/shell/commands/compact_rs.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/compact_rs.rb b/hbase-shell/src/main/ruby/shell/commands/compact_rs.rb
index 588b6fe..5f02944 100644
--- a/hbase-shell/src/main/ruby/shell/commands/compact_rs.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/compact_rs.rb
@@ -34,7 +34,7 @@ module Shell
       end
 
       def command(regionserver, major = false)
-        admin.compactRegionserver(regionserver, major)
+        admin.compact_regionserver(regionserver, major)
       end
     end
   end


[38/40] hbase git commit: HBASE-15583 Any HTableDescriptor we give out should be immutable

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
new file mode 100644
index 0000000..58a18ec
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -0,0 +1,256 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * TableDescriptor contains the details about an HBase table such as the descriptors of
+ * all the column families, is the table a catalog table, <code> -ROOT- </code> or
+ * <code> hbase:meta </code>, if the table is read only, the maximum size of the memstore,
+ * when the region split should occur, coprocessors associated with it etc...
+ */
+@InterfaceAudience.Public
+public interface TableDescriptor {
+
+  /**
+   * Returns an array all the {@link HColumnDescriptor} of the column families
+   * of the table.
+   *
+   * @return Array of all the HColumnDescriptors of the current table
+   *
+   * @see #getFamilies()
+   */
+  HColumnDescriptor[] getColumnFamilies();
+
+  /**
+   * Returns the count of the column families of the table.
+   *
+   * @return Count of column families of the table
+   */
+  int getColumnFamilyCount();
+
+  /**
+   * Getter for fetching an unmodifiable map.
+   *
+   * @return an unmodifiable map
+   */
+  Map<String, String> getConfiguration();
+
+  /**
+   * Getter for accessing the configuration value by key
+   *
+   * @param key the key whose associated value is to be returned
+   * @return the value to which the specified key is mapped, or {@code null} if
+   * this map contains no mapping for the key
+   */
+  String getConfigurationValue(String key);
+
+  /**
+   * Return the list of attached co-processor represented by their name
+   * className
+   *
+   * @return The list of co-processors classNames
+   */
+  Collection<String> getCoprocessors();
+
+  /**
+   * Returns the durability setting for the table.
+   *
+   * @return durability setting for the table.
+   */
+  Durability getDurability();
+
+  /**
+   * Returns an unmodifiable collection of all the {@link HColumnDescriptor} of
+   * all the column families of the table.
+   *
+   * @return Immutable collection of {@link HColumnDescriptor} of all the column
+   * families.
+   */
+  Collection<HColumnDescriptor> getFamilies();
+
+  /**
+   * Returns all the column family names of the current table. The map of
+   * TableDescriptor contains mapping of family name to HColumnDescriptors.
+   * This returns all the keys of the family map which represents the column
+   * family names of the table.
+   *
+   * @return Immutable sorted set of the keys of the families.
+   */
+  Set<byte[]> getFamiliesKeys();
+
+  /**
+   * Returns the HColumnDescriptor for a specific column family with name as
+   * specified by the parameter column.
+   *
+   * @param column Column family name
+   * @return Column descriptor for the passed family name or the family on
+   * passed in column.
+   */
+  HColumnDescriptor getFamily(final byte[] column);
+
+  /**
+   * This gets the class associated with the flush policy which determines the
+   * stores need to be flushed when flushing a region. The class used by default
+   * is defined in org.apache.hadoop.hbase.regionserver.FlushPolicy.
+   *
+   * @return the class name of the flush policy for this table. If this returns
+   * null, the default flush policy is used.
+   */
+  String getFlushPolicyClassName();
+
+  /**
+   * Returns the maximum size upto which a region can grow to after which a
+   * region split is triggered. The region size is represented by the size of
+   * the biggest store file in that region.
+   *
+   * @return max hregion size for table, -1 if not set.
+   */
+  long getMaxFileSize();
+
+  /**
+   * Returns the size of the memstore after which a flush to filesystem is
+   * triggered.
+   *
+   * @return memory cache flush size for each hregion, -1 if not set.
+   */
+  long getMemStoreFlushSize();
+
+  int getPriority();
+
+  /**
+   * @return Returns the configured replicas per region
+   */
+  int getRegionReplication();
+
+  /**
+   * This gets the class associated with the region split policy which
+   * determines when a region split should occur. The class used by default is
+   * defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+   *
+   * @return the class name of the region split policy for this table. If this
+   * returns null, the default split policy is used.
+   */
+  String getRegionSplitPolicyClassName();
+
+  /**
+   * Get the name of the table
+   *
+   * @return TableName
+   */
+  TableName getTableName();
+
+  @Deprecated
+  String getOwnerString();
+
+  /**
+   * Getter for accessing the metadata associated with the key
+   *
+   * @param key The key.
+   * @return The value.
+   */
+  byte[] getValue(byte[] key);
+
+  /**
+   * @return Getter for fetching an unmodifiable map.
+   */
+  Map<Bytes, Bytes> getValues();
+
+  /**
+   * Check if the table has an attached co-processor represented by the name
+   * className
+   *
+   * @param classNameToMatch - Class name of the co-processor
+   * @return true of the table has a co-processor className
+   */
+  boolean hasCoprocessor(String classNameToMatch);
+
+  /**
+   * Checks to see if this table contains the given column family
+   *
+   * @param familyName Family name or column name.
+   * @return true if the table contains the specified family name
+   */
+  boolean hasFamily(final byte[] familyName);
+
+  /**
+   * @return true if the read-replicas memstore replication is enabled.
+   */
+  boolean hasRegionMemstoreReplication();
+
+  /**
+   * @return true if there are at least one cf whose replication scope is
+   * serial.
+   */
+  boolean hasSerialReplicationScope();
+
+  /**
+   * Check if the compaction enable flag of the table is true. If flag is false
+   * then no minor/major compactions will be done in real.
+   *
+   * @return true if table compaction enabled
+   */
+  boolean isCompactionEnabled();
+
+  /**
+   * Checks if this table is <code> hbase:meta </code> region.
+   *
+   * @return true if this table is <code> hbase:meta </code> region
+   */
+  boolean isMetaRegion();
+
+  /**
+   * Checks if the table is a <code>hbase:meta</code> table
+   *
+   * @return true if table is <code> hbase:meta </code> region.
+   */
+  boolean isMetaTable();
+
+  /**
+   * Check if normalization enable flag of the table is true. If flag is false
+   * then no region normalizer won't attempt to normalize this table.
+   *
+   * @return true if region normalization is enabled for this table
+   */
+  boolean isNormalizationEnabled();
+
+  /**
+   * Check if the readOnly flag of the table is set. If the readOnly flag is set
+   * then the contents of the table can only be read from but not modified.
+   *
+   * @return true if all columns in the table should be read only
+   */
+  boolean isReadOnly();
+
+  /**
+   * Check if the descriptor represents a <code> -ROOT- </code> region.
+   *
+   * @return true if this is a <code> -ROOT- </code> region
+   */
+  boolean isRootRegion();
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
new file mode 100644
index 0000000..a372ced
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -0,0 +1,1639 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+@InterfaceAudience.Public
+public class TableDescriptorBuilder {
+
+  private static final Log LOG = LogFactory.getLog(TableDescriptorBuilder.class);
+
+  public static final String SPLIT_POLICY = "SPLIT_POLICY";
+
+  /**
+   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
+   * attribute which denotes the maximum size of the store file after which a
+   * region split occurs.
+   */
+  public static final String MAX_FILESIZE = "MAX_FILESIZE";
+  private static final Bytes MAX_FILESIZE_KEY
+          = new Bytes(Bytes.toBytes(MAX_FILESIZE));
+
+  public static final String OWNER = "OWNER";
+  public static final Bytes OWNER_KEY
+          = new Bytes(Bytes.toBytes(OWNER));
+
+  /**
+   * <em>INTERNAL</em> Used by rest interface to access this metadata attribute
+   * which denotes if the table is Read Only.
+   */
+  public static final String READONLY = "READONLY";
+  private static final Bytes READONLY_KEY
+          = new Bytes(Bytes.toBytes(READONLY));
+
+  /**
+   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
+   * attribute which denotes if the table is compaction enabled.
+   */
+  public static final String COMPACTION_ENABLED = "COMPACTION_ENABLED";
+  private static final Bytes COMPACTION_ENABLED_KEY
+          = new Bytes(Bytes.toBytes(COMPACTION_ENABLED));
+
+  /**
+   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
+   * attribute which represents the maximum size of the memstore after which its
+   * contents are flushed onto the disk.
+   */
+  public static final String MEMSTORE_FLUSHSIZE = "MEMSTORE_FLUSHSIZE";
+  private static final Bytes MEMSTORE_FLUSHSIZE_KEY
+          = new Bytes(Bytes.toBytes(MEMSTORE_FLUSHSIZE));
+
+  public static final String FLUSH_POLICY = "FLUSH_POLICY";
+
+  /**
+   * <em>INTERNAL</em> Used by rest interface to access this metadata attribute
+   * which denotes if the table is a -ROOT- region or not.
+   */
+  public static final String IS_ROOT = "IS_ROOT";
+  private static final Bytes IS_ROOT_KEY
+          = new Bytes(Bytes.toBytes(IS_ROOT));
+
+  /**
+   * <em>INTERNAL</em> Used by rest interface to access this metadata attribute
+   * which denotes if it is a catalog table, either <code> hbase:meta </code> or <code> -ROOT-
+   * </code>.
+   */
+  public static final String IS_META = "IS_META";
+  private static final Bytes IS_META_KEY
+          = new Bytes(Bytes.toBytes(IS_META));
+
+  /**
+   * <em>INTERNAL</em> {@link Durability} setting for the table.
+   */
+  public static final String DURABILITY = "DURABILITY";
+  private static final Bytes DURABILITY_KEY
+          = new Bytes(Bytes.toBytes("DURABILITY"));
+
+  /**
+   * <em>INTERNAL</em> number of region replicas for the table.
+   */
+  public static final String REGION_REPLICATION = "REGION_REPLICATION";
+  private static final Bytes REGION_REPLICATION_KEY
+          = new Bytes(Bytes.toBytes(REGION_REPLICATION));
+
+  /**
+   * <em>INTERNAL</em> flag to indicate whether or not the memstore should be
+   * replicated for read-replicas (CONSISTENCY =&gt; TIMELINE).
+   */
+  public static final String REGION_MEMSTORE_REPLICATION = "REGION_MEMSTORE_REPLICATION";
+  private static final Bytes REGION_MEMSTORE_REPLICATION_KEY
+          = new Bytes(Bytes.toBytes(REGION_MEMSTORE_REPLICATION));
+
+  /**
+   * <em>INTERNAL</em> Used by shell/rest interface to access this metadata
+   * attribute which denotes if the table should be treated by region
+   * normalizer.
+   */
+  public static final String NORMALIZATION_ENABLED = "NORMALIZATION_ENABLED";
+  private static final Bytes NORMALIZATION_ENABLED_KEY
+          = new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
+
+  /**
+   * Default durability for HTD is USE_DEFAULT, which defaults to HBase-global
+   * default value
+   */
+  private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
+
+  public static final String PRIORITY = "PRIORITY";
+  private static final Bytes PRIORITY_KEY
+          = new Bytes(Bytes.toBytes(PRIORITY));
+
+  /**
+   * Relative priority of the table used for rpc scheduling
+   */
+  private static final int DEFAULT_PRIORITY = HConstants.NORMAL_QOS;
+
+  /*
+     *  The below are ugly but better than creating them each time till we
+     *  replace booleans being saved as Strings with plain booleans.  Need a
+     *  migration script to do this.  TODO.
+   */
+  private static final Bytes FALSE
+          = new Bytes(Bytes.toBytes(Boolean.FALSE.toString()));
+
+  private static final Bytes TRUE
+          = new Bytes(Bytes.toBytes(Boolean.TRUE.toString()));
+
+  /**
+   * Constant that denotes whether the table is READONLY by default and is false
+   */
+  public static final boolean DEFAULT_READONLY = false;
+
+  /**
+   * Constant that denotes whether the table is compaction enabled by default
+   */
+  public static final boolean DEFAULT_COMPACTION_ENABLED = true;
+
+  /**
+   * Constant that denotes whether the table is normalized by default.
+   */
+  public static final boolean DEFAULT_NORMALIZATION_ENABLED = false;
+
+  /**
+   * Constant that denotes the maximum default size of the memstore after which
+   * the contents are flushed to the store files
+   */
+  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = 1024 * 1024 * 128L;
+
+  public static final int DEFAULT_REGION_REPLICATION = 1;
+
+  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true;
+
+  private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
+  private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
+
+  static {
+    DEFAULT_VALUES.put(MAX_FILESIZE,
+            String.valueOf(HConstants.DEFAULT_MAX_FILE_SIZE));
+    DEFAULT_VALUES.put(READONLY, String.valueOf(DEFAULT_READONLY));
+    DEFAULT_VALUES.put(MEMSTORE_FLUSHSIZE,
+            String.valueOf(DEFAULT_MEMSTORE_FLUSH_SIZE));
+    DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); //use the enum name
+    DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION));
+    DEFAULT_VALUES.put(NORMALIZATION_ENABLED, String.valueOf(DEFAULT_NORMALIZATION_ENABLED));
+    DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY));
+    DEFAULT_VALUES.keySet().stream()
+            .map(s -> new Bytes(Bytes.toBytes(s))).forEach(RESERVED_KEYWORDS::add);
+    RESERVED_KEYWORDS.add(IS_ROOT_KEY);
+    RESERVED_KEYWORDS.add(IS_META_KEY);
+  }
+
+  public final static String NAMESPACE_FAMILY_INFO = "info";
+  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO);
+  public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d");
+
+  /**
+   * Table descriptor for namespace table
+   */
+  public static final TableDescriptor NAMESPACE_TABLEDESC
+    = TableDescriptorBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME)
+                            .addFamily(new HColumnDescriptor(NAMESPACE_FAMILY_INFO)
+                              // Ten is arbitrary number.  Keep versions to help debugging.
+                              .setMaxVersions(10)
+                              .setInMemory(true)
+                              .setBlocksize(8 * 1024)
+                              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                              // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                              // e.g. if using CombinedBlockCache (BucketCache).
+                              .setCacheDataInL1(true))
+                            .doBuild();
+  private final ModifyableTableDescriptor desc;
+
+  /**
+   * @param desc The table descriptor to serialize
+   * @return This instance serialized with pb with pb magic prefix
+   */
+  public static byte[] toByteArray(TableDescriptor desc) {
+    if (desc instanceof ModifyableTableDescriptor) {
+      return ((ModifyableTableDescriptor) desc).toByteArray();
+    }
+    // TODO: remove this if the HTableDescriptor is removed
+    if (desc instanceof HTableDescriptor) {
+      return ((HTableDescriptor) desc).toByteArray();
+    }
+    return new ModifyableTableDescriptor(desc).toByteArray();
+  }
+
+  /**
+   * The input should be created by {@link #toByteArray}.
+   * @param pbBytes A pb serialized TableDescriptor instance with pb magic prefix
+   * @return This instance serialized with pb with pb magic prefix
+   * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
+   */
+  public static TableDescriptorBuilder newBuilder(byte[] pbBytes) throws DeserializationException {
+    return new TableDescriptorBuilder(ModifyableTableDescriptor.parseFrom(pbBytes));
+  }
+
+  public static TableDescriptorBuilder newBuilder(final TableName name) {
+    return new TableDescriptorBuilder(name);
+  }
+
+  /**
+   * Copy all configuration, values, families, and name from the input.
+   * @param desc The desciptor to copy
+   * @return A clone of input
+   */
+  public static TableDescriptorBuilder newBuilder(final TableDescriptor desc) {
+    return new TableDescriptorBuilder(desc);
+  }
+
+  private TableDescriptorBuilder(final TableName name) {
+    this.desc = new ModifyableTableDescriptor(name);
+  }
+
+  private TableDescriptorBuilder(final TableDescriptor desc) {
+    this.desc = new ModifyableTableDescriptor(desc);
+  }
+
+  public TableDescriptorBuilder addCoprocessor(String className) throws IOException {
+    return addCoprocessor(className, null, Coprocessor.PRIORITY_USER, null);
+  }
+
+  public TableDescriptorBuilder addCoprocessor(String className, Path jarFilePath,
+          int priority, final Map<String, String> kvs) throws IOException {
+    desc.addCoprocessor(className, jarFilePath, priority, kvs);
+    return this;
+  }
+
+  public TableDescriptorBuilder addCoprocessorWithSpec(final String specStr) throws IOException {
+    desc.addCoprocessorWithSpec(specStr);
+    return this;
+  }
+
+  public TableDescriptorBuilder addFamily(final HColumnDescriptor family) {
+    desc.addFamily(family);
+    return this;
+  }
+
+  public TableDescriptorBuilder modifyFamily(final HColumnDescriptor family) {
+    desc.modifyFamily(family);
+    return this;
+  }
+
+  public TableDescriptorBuilder remove(Bytes key) {
+    desc.remove(key);
+    return this;
+  }
+
+  public TableDescriptorBuilder remove(byte[] key) {
+    desc.remove(key);
+    return this;
+  }
+
+  public TableDescriptorBuilder removeConfiguration(final String key) {
+    desc.removeConfiguration(key);
+    return this;
+  }
+
+  public TableDescriptorBuilder removeFamily(final byte[] column) {
+    desc.removeFamily(column);
+    return this;
+  }
+
+  public TableDescriptorBuilder removeCoprocessor(String className) {
+    desc.removeCoprocessor(className);
+    return this;
+  }
+
+  public TableDescriptorBuilder setCompactionEnabled(final boolean isEnable) {
+    desc.setCompactionEnabled(isEnable);
+    return this;
+  }
+
+  public TableDescriptorBuilder setConfiguration(String key, String value) {
+    desc.setConfiguration(key, value);
+    return this;
+  }
+
+  public TableDescriptorBuilder setDurability(Durability durability) {
+    desc.setDurability(durability);
+    return this;
+  }
+
+  public TableDescriptorBuilder setFlushPolicyClassName(String clazz) {
+    desc.setFlushPolicyClassName(clazz);
+    return this;
+  }
+
+  public TableDescriptorBuilder setMaxFileSize(long maxFileSize) {
+    desc.setMaxFileSize(maxFileSize);
+    return this;
+  }
+
+  public TableDescriptorBuilder setMemStoreFlushSize(long memstoreFlushSize) {
+    desc.setMemStoreFlushSize(memstoreFlushSize);
+    return this;
+  }
+
+  public TableDescriptorBuilder setNormalizationEnabled(final boolean isEnable) {
+    desc.setNormalizationEnabled(isEnable);
+    return this;
+  }
+
+  @Deprecated
+  public TableDescriptorBuilder setOwner(User owner) {
+    desc.setOwner(owner);
+    return this;
+  }
+
+  @Deprecated
+  public TableDescriptorBuilder setOwnerString(String ownerString) {
+    desc.setOwnerString(ownerString);
+    return this;
+  }
+
+  public TableDescriptorBuilder setPriority(int priority) {
+    desc.setPriority(priority);
+    return this;
+  }
+
+  public TableDescriptorBuilder setReadOnly(final boolean readOnly) {
+    desc.setReadOnly(readOnly);
+    return this;
+  }
+
+  public TableDescriptorBuilder setRegionMemstoreReplication(boolean memstoreReplication) {
+    desc.setRegionMemstoreReplication(memstoreReplication);
+    return this;
+  }
+
+  public TableDescriptorBuilder setRegionReplication(int regionReplication) {
+    desc.setRegionReplication(regionReplication);
+    return this;
+  }
+
+  public TableDescriptorBuilder setRegionSplitPolicyClassName(String clazz) {
+    desc.setRegionSplitPolicyClassName(clazz);
+    return this;
+  }
+
+  public TableDescriptorBuilder setValue(final Bytes key, final Bytes value) {
+    desc.setValue(key, value);
+    return this;
+  }
+
+  public TableDescriptorBuilder setValue(final byte[] key, final byte[] value) {
+    desc.setValue(key, value);
+    return this;
+  }
+
+  // TODO: replaced the HTableDescriptor by TableDescriptor
+  public HTableDescriptor build() {
+    return new HTableDescriptor(desc);
+  }
+
+  // TODO: remove this in HBase 3.0.0.
+  private TableDescriptor doBuild() {
+    return new ModifyableTableDescriptor(desc);
+  }
+
+  /**
+   * ModifyableTableDescriptor contains the details about an HBase table such as the
+   * descriptors of all the column families, is the table a catalog table, <code> -ROOT-
+   * </code> or <code> hbase:meta </code>, if the table is read only, the
+   * maximum size of the memstore, when the region split should occur,
+   * coprocessors associated with it etc... TODO: make this private after
+   * removing the HTableDescriptor
+   */
+  @InterfaceAudience.Private
+  public static class ModifyableTableDescriptor
+          implements TableDescriptor, Comparable<ModifyableTableDescriptor> {
+
+    private final TableName name;
+
+    /**
+     * A map which holds the metadata information of the table. This metadata
+     * includes values like IS_ROOT, IS_META, SPLIT_POLICY, MAX_FILE_SIZE,
+     * READONLY, MEMSTORE_FLUSHSIZE etc...
+     */
+    private final Map<Bytes, Bytes> values = new HashMap<>();
+
+    /**
+     * A map which holds the configuration specific to the table. The keys of
+     * the map have the same names as config keys and override the defaults with
+     * table-specific settings. Example usage may be for compactions, etc.
+     */
+    private final Map<String, String> configuration = new HashMap<>();
+
+    /**
+     * Maps column family name to the respective HColumnDescriptors
+     */
+    private final Map<byte[], HColumnDescriptor> families
+            = new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
+
+    /**
+     * Construct a table descriptor specifying a TableName object
+     *
+     * @param name Table name.
+     * @see
+     * <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581
+     * HBASE: (HBASE-174) Un-openable tablename bug</a>
+     */
+    private ModifyableTableDescriptor(final TableName name) {
+      this(name, Collections.EMPTY_LIST, Collections.EMPTY_MAP, Collections.EMPTY_MAP);
+    }
+
+    /**
+     * Construct a table descriptor by cloning the descriptor passed as a
+     * parameter.
+     * <p>
+     * Makes a deep copy of the supplied descriptor.
+     * TODO: make this private after removing the HTableDescriptor
+     * @param desc The descriptor.
+     */
+    @InterfaceAudience.Private
+    protected ModifyableTableDescriptor(final TableDescriptor desc) {
+      this(desc.getTableName(), desc.getFamilies(), desc.getValues(), desc.getConfiguration());
+    }
+
+    // TODO: make this private after removing the HTableDescriptor
+    @InterfaceAudience.Private
+    public ModifyableTableDescriptor(final TableName name, final Collection<HColumnDescriptor> families,
+            Map<Bytes, Bytes> values, Map<String, String> configuration) {
+      this.name = name;
+      families.forEach(c -> this.families.put(c.getName(), new HColumnDescriptor(c)));
+      values.forEach(this.values::put);
+      configuration.forEach(this.configuration::put);
+      setMetaFlags(name);
+    }
+
+    /*
+     * Set meta flags on this table.
+     * IS_ROOT_KEY is set if its a -ROOT- table
+     * IS_META_KEY is set either if its a -ROOT- or a hbase:meta table
+     * Called by constructors.
+     * @param name
+     */
+    private void setMetaFlags(final TableName name) {
+      values.put(IS_META_KEY, isRootRegion()
+              || name.equals(TableName.META_TABLE_NAME) ? TRUE : FALSE);
+    }
+
+    /**
+     * Check if the descriptor represents a <code> -ROOT- </code> region.
+     *
+     * @return true if this is a <code> -ROOT- </code> region
+     */
+    @Override
+    public boolean isRootRegion() {
+      return isSomething(IS_ROOT_KEY, false);
+    }
+
+    /**
+     * Checks if this table is <code> hbase:meta </code> region.
+     *
+     * @return true if this table is <code> hbase:meta </code> region
+     */
+    @Override
+    public boolean isMetaRegion() {
+      return isSomething(IS_META_KEY, false);
+    }
+
+    private boolean isSomething(final Bytes key,
+            final boolean valueIfNull) {
+      byte[] value = getValue(key);
+      if (value != null) {
+        return Boolean.valueOf(Bytes.toString(value));
+      }
+      return valueIfNull;
+    }
+
+    /**
+     * Checks if the table is a <code>hbase:meta</code> table
+     *
+     * @return true if table is <code> hbase:meta </code> region.
+     */
+    @Override
+    public boolean isMetaTable() {
+      return isMetaRegion() && !isRootRegion();
+    }
+
+    /**
+     * Getter for accessing the metadata associated with the key
+     *
+     * @param key The key.
+     * @return The value.
+     * @see #values
+     */
+    @Override
+    public byte[] getValue(byte[] key) {
+      return getValue(new Bytes(key));
+    }
+
+    private byte[] getValue(final Bytes key) {
+      Bytes ibw = values.get(key);
+      if (ibw == null) {
+        return null;
+      }
+      return ibw.get();
+    }
+
+    /**
+     * Getter for accessing the metadata associated with the key
+     *
+     * @param key The key.
+     * @return The value.
+     * @see #values
+     */
+    public String getValue(String key) {
+      byte[] value = getValue(Bytes.toBytes(key));
+      if (value == null) {
+        return null;
+      }
+      return Bytes.toString(value);
+    }
+
+    /**
+     * Getter for fetching an unmodifiable {@link #values} map.
+     *
+     * @return unmodifiable map {@link #values}.
+     * @see #values
+     */
+    @Override
+    public Map<Bytes, Bytes> getValues() {
+      // shallow pointer copy
+      return Collections.unmodifiableMap(values);
+    }
+
+    /**
+     * Setter for storing metadata as a (key, value) pair in {@link #values} map
+     *
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     * @return the modifyable TD
+     * @see #values
+     */
+    public ModifyableTableDescriptor setValue(byte[] key, byte[] value) {
+      return setValue(toBytesOrNull(key, v -> v),
+              toBytesOrNull(value, v -> v));
+    }
+
+    /*
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     */
+    private ModifyableTableDescriptor setValue(final Bytes key,
+            final String value) {
+      return setValue(key, toBytesOrNull(value, Bytes::toBytes));
+    }
+
+    /*
+     * Setter for storing metadata as a (key, value) pair in {@link #values} map
+     *
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     */
+    public ModifyableTableDescriptor setValue(final Bytes key, final Bytes value) {
+      if (value == null) {
+        remove(key);
+      } else {
+        values.put(key, value);
+      }
+      return this;
+    }
+
+    /**
+     * Setter for storing metadata as a (key, value) pair in {@link #values} map
+     *
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     * @return the modifyable TD
+     * @see #values
+     */
+    public ModifyableTableDescriptor setValue(String key, String value) {
+      return setValue(toBytesOrNull(key, Bytes::toBytes),
+              toBytesOrNull(value, Bytes::toBytes));
+    }
+
+    private static <T> Bytes toBytesOrNull(T t, Function<T, byte[]> f) {
+      if (t == null) {
+        return null;
+      } else {
+        return new Bytes(f.apply(t));
+      }
+    }
+
+    /**
+     * Remove metadata represented by the key from the {@link #values} map
+     *
+     * @param key Key whose key and value we're to remove from TableDescriptor
+     * parameters.
+     */
+    public void remove(final String key) {
+      remove(new Bytes(Bytes.toBytes(key)));
+    }
+
+    /**
+     * Remove metadata represented by the key from the {@link #values} map
+     *
+     * @param key Key whose key and value we're to remove from TableDescriptor
+     * parameters.
+     */
+    public void remove(Bytes key) {
+      values.remove(key);
+    }
+
+    /**
+     * Remove metadata represented by the key from the {@link #values} map
+     *
+     * @param key Key whose key and value we're to remove from TableDescriptor
+     * parameters.
+     */
+    public void remove(final byte[] key) {
+      remove(new Bytes(key));
+    }
+
+    /**
+     * Check if the readOnly flag of the table is set. If the readOnly flag is
+     * set then the contents of the table can only be read from but not
+     * modified.
+     *
+     * @return true if all columns in the table should be read only
+     */
+    @Override
+    public boolean isReadOnly() {
+      return isSomething(READONLY_KEY, DEFAULT_READONLY);
+    }
+
+    /**
+     * Setting the table as read only sets all the columns in the table as read
+     * only. By default all tables are modifiable, but if the readOnly flag is
+     * set to true then the contents of the table can only be read but not
+     * modified.
+     *
+     * @param readOnly True if all of the columns in the table should be read
+     * only.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setReadOnly(final boolean readOnly) {
+      return setValue(READONLY_KEY, readOnly ? TRUE : FALSE);
+    }
+
+    /**
+     * Check if the compaction enable flag of the table is true. If flag is
+     * false then no minor/major compactions will be done in real.
+     *
+     * @return true if table compaction enabled
+     */
+    @Override
+    public boolean isCompactionEnabled() {
+      return isSomething(COMPACTION_ENABLED_KEY, DEFAULT_COMPACTION_ENABLED);
+    }
+
+    /**
+     * Setting the table compaction enable flag.
+     *
+     * @param isEnable True if enable compaction.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setCompactionEnabled(final boolean isEnable) {
+      return setValue(COMPACTION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    }
+
+    /**
+     * Check if normalization enable flag of the table is true. If flag is false
+     * then no region normalizer won't attempt to normalize this table.
+     *
+     * @return true if region normalization is enabled for this table
+     */
+    @Override
+    public boolean isNormalizationEnabled() {
+      return isSomething(NORMALIZATION_ENABLED_KEY, DEFAULT_NORMALIZATION_ENABLED);
+    }
+
+    /**
+     * Setting the table normalization enable flag.
+     *
+     * @param isEnable True if enable normalization.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setNormalizationEnabled(final boolean isEnable) {
+      return setValue(NORMALIZATION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    }
+
+    /**
+     * Sets the {@link Durability} setting for the table. This defaults to
+     * Durability.USE_DEFAULT.
+     *
+     * @param durability enum value
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setDurability(Durability durability) {
+      return setValue(DURABILITY_KEY, durability.name());
+    }
+
+    /**
+     * Returns the durability setting for the table.
+     *
+     * @return durability setting for the table.
+     */
+    @Override
+    public Durability getDurability() {
+      byte[] durabilityValue = getValue(DURABILITY_KEY);
+      if (durabilityValue == null) {
+        return DEFAULT_DURABLITY;
+      } else {
+        try {
+          return Durability.valueOf(Bytes.toString(durabilityValue));
+        } catch (IllegalArgumentException ex) {
+          LOG.warn("Received " + ex + " because Durability value for TableDescriptor"
+                  + " is not known. Durability:" + Bytes.toString(durabilityValue));
+          return DEFAULT_DURABLITY;
+        }
+      }
+    }
+
+    /**
+     * Get the name of the table
+     *
+     * @return TableName
+     */
+    @Override
+    public TableName getTableName() {
+      return name;
+    }
+
+    /**
+     * This sets the class associated with the region split policy which
+     * determines when a region split should occur. The class used by default is
+     * defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+     *
+     * @param clazz the class name
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setRegionSplitPolicyClassName(String clazz) {
+      return setValue(SPLIT_POLICY, clazz);
+    }
+
+    /**
+     * This gets the class associated with the region split policy which
+     * determines when a region split should occur. The class used by default is
+     * defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+     *
+     * @return the class name of the region split policy for this table. If this
+     * returns null, the default split policy is used.
+     */
+    @Override
+    public String getRegionSplitPolicyClassName() {
+      return getValue(SPLIT_POLICY);
+    }
+
+    /**
+     * Returns the maximum size upto which a region can grow to after which a
+     * region split is triggered. The region size is represented by the size of
+     * the biggest store file in that region.
+     *
+     * @return max hregion size for table, -1 if not set.
+     *
+     * @see #setMaxFileSize(long)
+     */
+    @Override
+    public long getMaxFileSize() {
+      byte[] value = getValue(MAX_FILESIZE_KEY);
+      if (value != null) {
+        return Long.parseLong(Bytes.toString(value));
+      }
+      return -1;
+    }
+
+    /**
+     * Sets the maximum size upto which a region can grow to after which a
+     * region split is triggered. The region size is represented by the size of
+     * the biggest store file in that region, i.e. If the biggest store file
+     * grows beyond the maxFileSize, then the region split is triggered. This
+     * defaults to a value of 256 MB.
+     * <p>
+     * This is not an absolute value and might vary. Assume that a single row
+     * exceeds the maxFileSize then the storeFileSize will be greater than
+     * maxFileSize since a single row cannot be split across multiple regions
+     * </p>
+     *
+     * @param maxFileSize The maximum file size that a store file can grow to
+     * before a split is triggered.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setMaxFileSize(long maxFileSize) {
+      return setValue(MAX_FILESIZE_KEY, Long.toString(maxFileSize));
+    }
+
+    /**
+     * Returns the size of the memstore after which a flush to filesystem is
+     * triggered.
+     *
+     * @return memory cache flush size for each hregion, -1 if not set.
+     *
+     * @see #setMemStoreFlushSize(long)
+     */
+    @Override
+    public long getMemStoreFlushSize() {
+      byte[] value = getValue(MEMSTORE_FLUSHSIZE_KEY);
+      if (value != null) {
+        return Long.parseLong(Bytes.toString(value));
+      }
+      return -1;
+    }
+
+    /**
+     * Represents the maximum size of the memstore after which the contents of
+     * the memstore are flushed to the filesystem. This defaults to a size of 64
+     * MB.
+     *
+     * @param memstoreFlushSize memory cache flush size for each hregion
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
+      return setValue(MEMSTORE_FLUSHSIZE_KEY, Long.toString(memstoreFlushSize));
+    }
+
+    /**
+     * This sets the class associated with the flush policy which determines
+     * determines the stores need to be flushed when flushing a region. The
+     * class used by default is defined in
+     * org.apache.hadoop.hbase.regionserver.FlushPolicy.
+     *
+     * @param clazz the class name
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setFlushPolicyClassName(String clazz) {
+      return setValue(FLUSH_POLICY, clazz);
+    }
+
+    /**
+     * This gets the class associated with the flush policy which determines the
+     * stores need to be flushed when flushing a region. The class used by
+     * default is defined in org.apache.hadoop.hbase.regionserver.FlushPolicy.
+     *
+     * @return the class name of the flush policy for this table. If this
+     * returns null, the default flush policy is used.
+     */
+    @Override
+    public String getFlushPolicyClassName() {
+      return getValue(FLUSH_POLICY);
+    }
+
+    /**
+     * Adds a column family. For the updating purpose please use
+     * {@link #modifyFamily(HColumnDescriptor)} instead.
+     *
+     * @param family HColumnDescriptor of family to add.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addFamily(final HColumnDescriptor family) {
+      if (family.getName() == null || family.getName().length <= 0) {
+        throw new IllegalArgumentException("Family name cannot be null or empty");
+      }
+      if (hasFamily(family.getName())) {
+        throw new IllegalArgumentException("Family '"
+                + family.getNameAsString() + "' already exists so cannot be added");
+      }
+      return setFamily(family);
+    }
+
+    /**
+     * Modifies the existing column family.
+     *
+     * @param family HColumnDescriptor of family to update
+     * @return this (for chained invocation)
+     */
+    public ModifyableTableDescriptor modifyFamily(final HColumnDescriptor family) {
+      if (family.getName() == null || family.getName().length <= 0) {
+        throw new IllegalArgumentException("Family name cannot be null or empty");
+      }
+      if (!hasFamily(family.getName())) {
+        throw new IllegalArgumentException("Column family '" + family.getNameAsString()
+                + "' does not exist");
+      }
+      return setFamily(family);
+    }
+
+    // TODO: make this private after removing the UnmodifyableTableDescriptor
+    protected ModifyableTableDescriptor setFamily(HColumnDescriptor family) {
+      families.put(family.getName(), family);
+      return this;
+    }
+
+    /**
+     * Checks to see if this table contains the given column family
+     *
+     * @param familyName Family name or column name.
+     * @return true if the table contains the specified family name
+     */
+    @Override
+    public boolean hasFamily(final byte[] familyName) {
+      return families.containsKey(familyName);
+    }
+
+    /**
+     * @return Name of this table and then a map of all of the column family descriptors.
+     */
+    @Override
+    public String toString() {
+      StringBuilder s = new StringBuilder();
+      s.append('\'').append(Bytes.toString(name.getName())).append('\'');
+      s.append(getValues(true));
+      families.values().forEach(f -> s.append(", ").append(f));
+      return s.toString();
+    }
+
+    /**
+     * @return Name of this table and then a map of all of the column family
+     * descriptors (with only the non-default column family attributes)
+     */
+    public String toStringCustomizedValues() {
+      StringBuilder s = new StringBuilder();
+      s.append('\'').append(Bytes.toString(name.getName())).append('\'');
+      s.append(getValues(false));
+      families.values().forEach(hcd -> s.append(", ").append(hcd.toStringCustomizedValues()));
+      return s.toString();
+    }
+
+    /**
+     * @return map of all table attributes formatted into string.
+     */
+    public String toStringTableAttributes() {
+      return getValues(true).toString();
+    }
+
+    private StringBuilder getValues(boolean printDefaults) {
+      StringBuilder s = new StringBuilder();
+
+      // step 1: set partitioning and pruning
+      Set<Bytes> reservedKeys = new TreeSet<>();
+      Set<Bytes> userKeys = new TreeSet<>();
+      for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
+        if (entry.getKey() == null || entry.getKey().get() == null) {
+          continue;
+        }
+        String key = Bytes.toString(entry.getKey().get());
+        // in this section, print out reserved keywords + coprocessor info
+        if (!RESERVED_KEYWORDS.contains(entry.getKey()) && !key.startsWith("coprocessor$")) {
+          userKeys.add(entry.getKey());
+          continue;
+        }
+        // only print out IS_ROOT/IS_META if true
+        String value = Bytes.toString(entry.getValue().get());
+        if (key.equalsIgnoreCase(IS_ROOT) || key.equalsIgnoreCase(IS_META)) {
+          if (Boolean.valueOf(value) == false) {
+            continue;
+          }
+        }
+        // see if a reserved key is a default value. may not want to print it out
+        if (printDefaults
+                || !DEFAULT_VALUES.containsKey(key)
+                || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
+          reservedKeys.add(entry.getKey());
+        }
+      }
+
+      // early exit optimization
+      boolean hasAttributes = !reservedKeys.isEmpty() || !userKeys.isEmpty();
+      if (!hasAttributes && configuration.isEmpty()) {
+        return s;
+      }
+
+      s.append(", {");
+      // step 2: printing attributes
+      if (hasAttributes) {
+        s.append("TABLE_ATTRIBUTES => {");
+
+        // print all reserved keys first
+        boolean printCommaForAttr = false;
+        for (Bytes k : reservedKeys) {
+          String key = Bytes.toString(k.get());
+          String value = Bytes.toStringBinary(values.get(k).get());
+          if (printCommaForAttr) {
+            s.append(", ");
+          }
+          printCommaForAttr = true;
+          s.append(key);
+          s.append(" => ");
+          s.append('\'').append(value).append('\'');
+        }
+
+        if (!userKeys.isEmpty()) {
+          // print all non-reserved, advanced config keys as a separate subset
+          if (printCommaForAttr) {
+            s.append(", ");
+          }
+          s.append(HConstants.METADATA).append(" => ");
+          s.append("{");
+          boolean printCommaForCfg = false;
+          for (Bytes k : userKeys) {
+            String key = Bytes.toString(k.get());
+            String value = Bytes.toStringBinary(values.get(k).get());
+            if (printCommaForCfg) {
+              s.append(", ");
+            }
+            printCommaForCfg = true;
+            s.append('\'').append(key).append('\'');
+            s.append(" => ");
+            s.append('\'').append(value).append('\'');
+          }
+          s.append("}");
+        }
+      }
+
+      // step 3: printing all configuration:
+      if (!configuration.isEmpty()) {
+        if (hasAttributes) {
+          s.append(", ");
+        }
+        s.append(HConstants.CONFIGURATION).append(" => ");
+        s.append('{');
+        boolean printCommaForConfig = false;
+        for (Map.Entry<String, String> e : configuration.entrySet()) {
+          if (printCommaForConfig) {
+            s.append(", ");
+          }
+          printCommaForConfig = true;
+          s.append('\'').append(e.getKey()).append('\'');
+          s.append(" => ");
+          s.append('\'').append(e.getValue()).append('\'');
+        }
+        s.append("}");
+      }
+      s.append("}"); // end METHOD
+      return s;
+    }
+
+    /**
+     * Compare the contents of the descriptor with another one passed as a
+     * parameter. Checks if the obj passed is an instance of ModifyableTableDescriptor,
+     * if yes then the contents of the descriptors are compared.
+     *
+     * @return true if the contents of the the two descriptors exactly match
+     *
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (!(obj instanceof ModifyableTableDescriptor)) {
+        return false;
+      }
+      return compareTo((ModifyableTableDescriptor) obj) == 0;
+    }
+
+    /**
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+      int result = this.name.hashCode();
+      if (this.families.size() > 0) {
+        for (HColumnDescriptor e : this.families.values()) {
+          result ^= e.hashCode();
+        }
+      }
+      result ^= values.hashCode();
+      result ^= configuration.hashCode();
+      return result;
+    }
+
+    // Comparable
+    /**
+     * Compares the descriptor with another descriptor which is passed as a
+     * parameter. This compares the content of the two descriptors and not the
+     * reference.
+     *
+     * @param other The MTD to compare
+     * @return 0 if the contents of the descriptors are exactly matching, 1 if
+     * there is a mismatch in the contents
+     */
+    @Override
+    public int compareTo(final ModifyableTableDescriptor other) {
+      int result = this.name.compareTo(other.name);
+      if (result == 0) {
+        result = families.size() - other.families.size();
+      }
+      if (result == 0 && families.size() != other.families.size()) {
+        result = Integer.valueOf(families.size()).compareTo(other.families.size());
+      }
+      if (result == 0) {
+        for (Iterator<HColumnDescriptor> it = families.values().iterator(),
+                it2 = other.families.values().iterator(); it.hasNext();) {
+          result = it.next().compareTo(it2.next());
+          if (result != 0) {
+            break;
+          }
+        }
+      }
+      if (result == 0) {
+        // punt on comparison for ordering, just calculate difference
+        result = this.values.hashCode() - other.values.hashCode();
+        if (result < 0) {
+          result = -1;
+        } else if (result > 0) {
+          result = 1;
+        }
+      }
+      if (result == 0) {
+        result = this.configuration.hashCode() - other.configuration.hashCode();
+        if (result < 0) {
+          result = -1;
+        } else if (result > 0) {
+          result = 1;
+        }
+      }
+      return result;
+    }
+
+    /**
+     * Returns an unmodifiable collection of all the {@link HColumnDescriptor}
+     * of all the column families of the table.
+     *
+     * @return Immutable collection of {@link HColumnDescriptor} of all the
+     * column families.
+     */
+    @Override
+    public Collection<HColumnDescriptor> getFamilies() {
+      return Collections.unmodifiableCollection(this.families.values());
+    }
+
+    /**
+     * Return true if there are at least one cf whose replication scope is
+     * serial.
+     */
+    @Override
+    public boolean hasSerialReplicationScope() {
+      return getFamilies()
+              .stream()
+              .anyMatch(column -> column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL);
+    }
+
+    /**
+     * Returns the configured replicas per region
+     */
+    @Override
+    public int getRegionReplication() {
+      return getIntValue(REGION_REPLICATION_KEY, DEFAULT_REGION_REPLICATION);
+    }
+
+    private int getIntValue(Bytes key, int defaultVal) {
+      byte[] val = getValue(key);
+      if (val == null || val.length == 0) {
+        return defaultVal;
+      }
+      return Integer.parseInt(Bytes.toString(val));
+    }
+
+    /**
+     * Sets the number of replicas per region.
+     *
+     * @param regionReplication the replication factor per region
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setRegionReplication(int regionReplication) {
+      return setValue(REGION_REPLICATION_KEY,
+              new Bytes(Bytes.toBytes(Integer.toString(regionReplication))));
+    }
+
+    /**
+     * @return true if the read-replicas memstore replication is enabled.
+     */
+    @Override
+    public boolean hasRegionMemstoreReplication() {
+      return isSomething(REGION_MEMSTORE_REPLICATION_KEY, DEFAULT_REGION_MEMSTORE_REPLICATION);
+    }
+
+    /**
+     * Enable or Disable the memstore replication from the primary region to the
+     * replicas. The replication will be used only for meta operations (e.g.
+     * flush, compaction, ...)
+     *
+     * @param memstoreReplication true if the new data written to the primary
+     * region should be replicated. false if the secondaries can tollerate to
+     * have new data only when the primary flushes the memstore.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
+      setValue(REGION_MEMSTORE_REPLICATION_KEY, memstoreReplication ? TRUE : FALSE);
+      // If the memstore replication is setup, we do not have to wait for observing a flush event
+      // from primary before starting to serve reads, because gaps from replication is not applicable
+      return setConfiguration(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY,
+              Boolean.toString(memstoreReplication));
+    }
+
+    public ModifyableTableDescriptor setPriority(int priority) {
+      return setValue(PRIORITY_KEY, Integer.toString(priority));
+    }
+
+    @Override
+    public int getPriority() {
+      return getIntValue(PRIORITY_KEY, DEFAULT_PRIORITY);
+    }
+
+    /**
+     * Returns all the column family names of the current table. The map of
+     * TableDescriptor contains mapping of family name to HColumnDescriptors.
+     * This returns all the keys of the family map which represents the column
+     * family names of the table.
+     *
+     * @return Immutable sorted set of the keys of the families.
+     */
+    @Override
+    public Set<byte[]> getFamiliesKeys() {
+      return Collections.unmodifiableSet(this.families.keySet());
+    }
+
+    /**
+     * Returns the count of the column families of the table.
+     *
+     * @return Count of column families of the table
+     */
+    @Override
+    public int getColumnFamilyCount() {
+      return families.size();
+    }
+
+    /**
+     * Returns an array all the {@link HColumnDescriptor} of the column families
+     * of the table.
+     *
+     * @return Array of all the HColumnDescriptors of the current table
+     *
+     * @see #getFamilies()
+     */
+    @Override
+    public HColumnDescriptor[] getColumnFamilies() {
+      Collection<HColumnDescriptor> hColumnDescriptors = getFamilies();
+      return hColumnDescriptors.toArray(new HColumnDescriptor[hColumnDescriptors.size()]);
+    }
+
+    /**
+     * Returns the HColumnDescriptor for a specific column family with name as
+     * specified by the parameter column.
+     *
+     * @param column Column family name
+     * @return Column descriptor for the passed family name or the family on
+     * passed in column.
+     */
+    @Override
+    public HColumnDescriptor getFamily(final byte[] column) {
+      return this.families.get(column);
+    }
+
+    /**
+     * Removes the HColumnDescriptor with name specified by the parameter column
+     * from the table descriptor
+     *
+     * @param column Name of the column family to be removed.
+     * @return Column descriptor for the passed family name or the family on
+     * passed in column.
+     */
+    public HColumnDescriptor removeFamily(final byte[] column) {
+      return this.families.remove(column);
+    }
+
+    /**
+     * Add a table coprocessor to this table. The coprocessor type must be
+     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't
+     * check if the class can be loaded or not. Whether a coprocessor is
+     * loadable or not will be determined when a region is opened.
+     *
+     * @param className Full class name.
+     * @throws IOException
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addCoprocessor(String className) throws IOException {
+      return addCoprocessor(className, null, Coprocessor.PRIORITY_USER, null);
+    }
+
+    /**
+     * Add a table coprocessor to this table. The coprocessor type must be
+     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't
+     * check if the class can be loaded or not. Whether a coprocessor is
+     * loadable or not will be determined when a region is opened.
+     *
+     * @param jarFilePath Path of the jar file. If it's null, the class will be
+     * loaded from default classloader.
+     * @param className Full class name.
+     * @param priority Priority
+     * @param kvs Arbitrary key-value parameter pairs passed into the
+     * coprocessor.
+     * @throws IOException
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addCoprocessor(String className, Path jarFilePath,
+            int priority, final Map<String, String> kvs)
+            throws IOException {
+      checkHasCoprocessor(className);
+
+      // Validate parameter kvs and then add key/values to kvString.
+      StringBuilder kvString = new StringBuilder();
+      if (kvs != null) {
+        for (Map.Entry<String, String> e : kvs.entrySet()) {
+          if (!e.getKey().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_KEY_PATTERN)) {
+            throw new IOException("Illegal parameter key = " + e.getKey());
+          }
+          if (!e.getValue().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN)) {
+            throw new IOException("Illegal parameter (" + e.getKey()
+                    + ") value = " + e.getValue());
+          }
+          if (kvString.length() != 0) {
+            kvString.append(',');
+          }
+          kvString.append(e.getKey());
+          kvString.append('=');
+          kvString.append(e.getValue());
+        }
+      }
+
+      String value = ((jarFilePath == null) ? "" : jarFilePath.toString())
+              + "|" + className + "|" + Integer.toString(priority) + "|"
+              + kvString.toString();
+      return addCoprocessorToMap(value);
+    }
+
+    /**
+     * Add a table coprocessor to this table. The coprocessor type must be
+     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't
+     * check if the class can be loaded or not. Whether a coprocessor is
+     * loadable or not will be determined when a region is opened.
+     *
+     * @param specStr The Coprocessor specification all in in one String
+     * formatted so matches {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
+     * @throws IOException
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
+      String className = getCoprocessorClassNameFromSpecStr(specStr);
+      if (className == null) {
+        throw new IllegalArgumentException("Format does not match "
+                + HConstants.CP_HTD_ATTR_VALUE_PATTERN + ": " + specStr);
+      }
+      checkHasCoprocessor(className);
+      return addCoprocessorToMap(specStr);
+    }
+
+    private void checkHasCoprocessor(final String className) throws IOException {
+      if (hasCoprocessor(className)) {
+        throw new IOException("Coprocessor " + className + " already exists.");
+      }
+    }
+
+    /**
+     * Add coprocessor to values Map
+     *
+     * @param specStr The Coprocessor specification all in in one String
+     * formatted so matches {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
+     * @return Returns <code>this</code>
+     */
+    private ModifyableTableDescriptor addCoprocessorToMap(final String specStr) {
+      if (specStr == null) {
+        return this;
+      }
+      // generate a coprocessor key
+      int maxCoprocessorNumber = 0;
+      Matcher keyMatcher;
+      for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
+        keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        maxCoprocessorNumber = Math.max(Integer.parseInt(keyMatcher.group(1)), maxCoprocessorNumber);
+      }
+      maxCoprocessorNumber++;
+      String key = "coprocessor$" + Integer.toString(maxCoprocessorNumber);
+      return setValue(new Bytes(Bytes.toBytes(key)), new Bytes(Bytes.toBytes(specStr)));
+    }
+
+    /**
+     * Check if the table has an attached co-processor represented by the name
+     * className
+     *
+     * @param classNameToMatch - Class name of the co-processor
+     * @return true of the table has a co-processor className
+     */
+    @Override
+    public boolean hasCoprocessor(String classNameToMatch) {
+      Matcher keyMatcher;
+      for (Map.Entry<Bytes, Bytes> e
+              : this.values.entrySet()) {
+        keyMatcher
+                = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(
+                        Bytes.toString(e.getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
+        if (className == null) {
+          continue;
+        }
+        if (className.equals(classNameToMatch.trim())) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    /**
+     * Return the list of attached co-processor represented by their name
+     * className
+     *
+     * @return The list of co-processors classNames
+     */
+    @Override
+    public List<String> getCoprocessors() {
+      List<String> result = new ArrayList<>(this.values.entrySet().size());
+      Matcher keyMatcher;
+      for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
+        keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
+        if (className == null) {
+          continue;
+        }
+        result.add(className); // classname is the 2nd field
+      }
+      return result;
+    }
+
+    /**
+     * @param spec String formatted as per
+     * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
+     * @return Class parsed from passed in <code>spec</code> or null if no match
+     * or classpath found
+     */
+    private static String getCoprocessorClassNameFromSpecStr(final String spec) {
+      Matcher matcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(spec);
+      // Classname is the 2nd field
+      return matcher != null && matcher.matches() ? matcher.group(2).trim() : null;
+    }
+
+    /**
+     * Remove a coprocessor from those set on the table
+     *
+     * @param className Class name of the co-processor
+     */
+    public void removeCoprocessor(String className) {
+      Bytes match = null;
+      Matcher keyMatcher;
+      Matcher valueMatcher;
+      for (Map.Entry<Bytes, Bytes> e : this.values
+              .entrySet()) {
+        keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e
+                .getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes
+                .toString(e.getValue().get()));
+        if (!valueMatcher.matches()) {
+          continue;
+        }
+        // get className and compare
+        String clazz = valueMatcher.group(2).trim(); // classname is the 2nd field
+        // remove the CP if it is present
+        if (clazz.equals(className.trim())) {
+          match = e.getKey();
+          break;
+        }
+      }
+      // if we found a match, remove it
+      if (match != null) {
+        remove(match);
+      }
+    }
+
+    @Deprecated
+    public ModifyableTableDescriptor setOwner(User owner) {
+      return setOwnerString(owner != null ? owner.getShortName() : null);
+    }
+
+    // used by admin.rb:alter(table_name,*args) to update owner.
+    @Deprecated
+    public ModifyableTableDescriptor setOwnerString(String ownerString) {
+      if (ownerString != null) {
+        setValue(OWNER_KEY, ownerString);
+      } else {
+        remove(OWNER_KEY);
+      }
+      return this;
+    }
+
+    @Override
+    @Deprecated
+    public String getOwnerString() {
+      if (getValue(OWNER_KEY) != null) {
+        return Bytes.toString(getValue(OWNER_KEY));
+      }
+      // Note that every table should have an owner (i.e. should have OWNER_KEY set).
+      // hbase:meta and -ROOT- should return system user as owner, not null (see
+      // MasterFileSystem.java:bootstrap()).
+      return null;
+    }
+
+    public byte[] toByteArray() {
+      return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
+    }
+
+    /**
+     * @param bytes A pb serialized {@link ModifyableTableDescriptor} instance
+     * with pb magic prefix
+     * @return An instance of {@link ModifyableTableDescriptor} made from
+     * <code>bytes</code>
+     * @throws DeserializationException
+     * @see #toByteArray()
+     */
+    public static TableDescriptor parseFrom(final byte[] bytes)
+            throws DeserializationException {
+      if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
+        throw new DeserializationException("Expected PB encoded ModifyableTableDescriptor");
+      }
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      HBaseProtos.TableSchema.Builder builder = HBaseProtos.TableSchema.newBuilder();
+      try {
+        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+        return ProtobufUtil.convertToTableDesc(builder.build());
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+    }
+
+    /**
+     * Getter for accessing the configuration value by key
+     */
+    @Override
+    public String getConfigurationValue(String key) {
+      return configuration.get(key);
+    }
+
+    /**
+     * Getter for fetching an unmodifiable {@link #configuration} map.
+     */
+    @Override
+    public Map<String, String> getConfiguration() {
+      // shallow pointer copy
+      return Collections.unmodifiableMap(configuration);
+    }
+
+    /**
+     * Setter for storing a configuration setting in {@link #configuration} map.
+     *
+     * @param key Config key. Same as XML config key e.g.
+     * hbase.something.or.other.
+     * @param value String value. If null, removes the setting.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setConfiguration(String key, String value) {
+      if (value == null) {
+        removeConfiguration(key);
+      } else {
+        configuration.put(key, value);
+      }
+      return this;
+    }
+
+    /**
+     * Remove a config setting represented by the key from the
+     * {@link #configuration} map
+     * @param key Config key.
+     */
+    public void removeConfiguration(final String key) {
+      configuration.remove(key);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/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
deleted file mode 100644
index b5f5ae9..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Read-only table descriptor.
- */
-@InterfaceAudience.Public
-public class UnmodifyableHTableDescriptor extends HTableDescriptor {
-  /**
-   * Default constructor.
-   * @deprecated  As of release 2.0.0. This will be removed in HBase 3.0.0.
-   *              Use {@link #UnmodifyableHTableDescriptor(HTableDescriptor)}.
-   */
-  @Deprecated
-  public UnmodifyableHTableDescriptor() {
-    super();
-  }
-
-  /*
-   * Create an unmodifyable copy of an HTableDescriptor
-   * @param desc
-   */
-  UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
-    super(desc.getTableName(), getUnmodifyableFamilies(desc), desc.getValues());
-  }
-
-
-  /*
-   * @param desc
-   * @return Families as unmodifiable array.
-   */
-  private static HColumnDescriptor[] getUnmodifyableFamilies(
-      final HTableDescriptor desc) {
-    HColumnDescriptor [] f = new HColumnDescriptor[desc.getFamilies().size()];
-    int i = 0;
-    for (HColumnDescriptor c: desc.getFamilies()) {
-      f[i++] = c;
-    }
-    return f;
-  }
-
-  /**
-   * Does NOT add a column family. This object is immutable
-   * @param family HColumnDescriptor of familyto add.
-   */
-  @Override
-  public UnmodifyableHTableDescriptor addFamily(final HColumnDescriptor family) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  @Override
-  public UnmodifyableHTableDescriptor modifyFamily(HColumnDescriptor family) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @param column
-   * @return Column descriptor for the passed family name or the family on
-   * passed in column.
-   */
-  @Override
-  public HColumnDescriptor removeFamily(final byte [] column) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setReadOnly(boolean)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setReadOnly(boolean readOnly) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setValue(byte[], byte[])
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setValue(byte[] key, byte[] value) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setValue(java.lang.String, java.lang.String)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setValue(String key, String value) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setMaxFileSize(long)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setMaxFileSize(long maxFileSize) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setMemStoreFlushSize(long)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 04ce040..d4c4231 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -35,6 +35,7 @@ import java.util.NavigableSet;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -67,6 +68,7 @@ import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
@@ -76,6 +78,8 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.client.SnapshotType;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -420,19 +424,34 @@ public final class ProtobufUtil {
    * Get HTableDescriptor[] from GetTableDescriptorsResponse protobuf
    *
    * @param proto the GetTableDescriptorsResponse
-   * @return HTableDescriptor[]
+   * @return a immutable HTableDescriptor array
+   * @deprecated Use {@link #getTableDescriptorArray} after removing the HTableDescriptor
    */
+  @Deprecated
   public static HTableDescriptor[] getHTableDescriptorArray(GetTableDescriptorsResponse proto) {
     if (proto == null) return null;
 
     HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
     for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
-      ret[i] = convertToHTableDesc(proto.getTableSchema(i));
+      ret[i] = new ImmutableHTableDescriptor(convertToHTableDesc(proto.getTableSchema(i)));
     }
     return ret;
   }
 
   /**
+   * Get TableDescriptor[] from GetTableDescriptorsResponse protobuf
+   *
+   * @param proto the GetTableDescriptorsResponse
+   * @return TableDescriptor[]
+   */
+  public static TableDescriptor[] getTableDescriptorArray(GetTableDescriptorsResponse proto) {
+    if (proto == null) return new TableDescriptor[0];
+    return proto.getTableSchemaList()
+                .stream()
+                .map(ProtobufUtil::convertToTableDesc)
+                .toArray(size -> new TableDescriptor[size]);
+  }
+  /**
    * get the split keys in form "byte [][]" from a CreateTableRequest proto
    *
    * @param proto the CreateTableRequest
@@ -2850,7 +2869,7 @@ public final class ProtobufUtil {
    * @param htd the HTableDescriptor
    * @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
    */
-  public static TableSchema convertToTableSchema(HTableDescriptor htd) {
+  public static TableSchema convertToTableSchema(TableDescriptor htd) {
     TableSchema.Builder builder = TableSchema.newBuilder();
     builder.setTableName(toProtoTableName(htd.getTableName()));
     for (Map.Entry<Bytes, Bytes> e : htd.getValues().entrySet()) {
@@ -2875,7 +2894,9 @@ public final class ProtobufUtil {
    * Converts a TableSchema to HTableDescriptor
    * @param ts A pb TableSchema instance.
    * @return An {@link HTableDescriptor} made from the passed in pb <code>ts</code>.
+   * @deprecated Use {@link #convertToTableDesc} after removing the HTableDescriptor
    */
+  @Deprecated
   public static HTableDescriptor convertToHTableDesc(final TableSchema ts) {
     List<ColumnFamilySchema> list = ts.getColumnFamiliesList();
     HColumnDescriptor [] hcds = new HColumnDescriptor[list.size()];
@@ -2897,6 +2918,25 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Converts a TableSchema to TableDescriptor
+   * @param ts A pb TableSchema instance.
+   * @return An {@link TableDescriptor} made from the passed in pb <code>ts</code>.
+   */
+  public static TableDescriptor convertToTableDesc(final TableSchema ts) {
+    TableDescriptorBuilder builder
+      = TableDescriptorBuilder.newBuilder(ProtobufUtil.toTableName(ts.getTableName()));
+    ts.getColumnFamiliesList()
+      .stream()
+      .map(ProtobufUtil::convertToHColumnDesc)
+      .forEach(builder::addFamily);
+    ts.getAttributesList()
+      .forEach(a -> builder.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray()));
+    ts.getConfigurationList()
+      .forEach(a -> builder.setConfiguration(a.getName(), a.getValue()));
+    return builder.build();
+  }
+
+  /**
    * Creates {@link CompactionState} from
    * {@link org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState}
    * state

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index a513d66..366e050 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -1244,7 +1245,7 @@ public final class RequestConverter {
    * @return a CreateTableRequest
    */
   public static CreateTableRequest buildCreateTableRequest(
-      final HTableDescriptor hTableDesc,
+      final TableDescriptor hTableDesc,
       final byte [][] splitKeys,
       final long nonceGroup,
       final long nonce) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java
new file mode 100644
index 0000000..91ef72a
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.BuilderStyleTest;
+import org.apache.hadoop.hbase.util.Bytes;
+import static org.junit.Assert.fail;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category({ClientTests.class, SmallTests.class})
+public class TestImmutableHTableDescriptor {
+  @Rule
+  public TestName name = new TestName();
+  private static final List<Consumer<ImmutableHTableDescriptor>> TEST_FUNCTION = Arrays.asList(
+    htd -> htd.setValue("a", "a"),
+    htd -> htd.setValue(Bytes.toBytes("a"), Bytes.toBytes("a")),
+    htd -> htd.setValue(new Bytes(Bytes.toBytes("a")), new Bytes(Bytes.toBytes("a"))),
+    htd -> htd.setCompactionEnabled(false),
+    htd -> htd.setConfiguration("aaa", "ccc"),
+    htd -> htd.setDurability(Durability.USE_DEFAULT),
+    htd -> htd.setFlushPolicyClassName("class"),
+    htd -> htd.setMaxFileSize(123),
+    htd -> htd.setMemStoreFlushSize(123123123),
+    htd -> htd.setNormalizationEnabled(false),
+    htd -> htd.setPriority(123),
+    htd -> htd.setReadOnly(true),
+    htd -> htd.setRegionMemstoreReplication(true),
+    htd -> htd.setRegionReplication(123),
+    htd -> htd.setRegionSplitPolicyClassName("class"),
+    htd -> htd.addFamily(new HColumnDescriptor(Bytes.toBytes("fm"))),
+    htd -> htd.remove(new Bytes(Bytes.toBytes("aaa"))),
+    htd -> htd.remove("aaa"),
+    htd -> htd.remove(Bytes.toBytes("aaa")),
+    htd -> htd.removeConfiguration("xxx"),
+    htd -> htd.removeFamily(Bytes.toBytes("fm")),
+    htd -> {
+      try {
+        htd.addCoprocessor("xxx");
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  );
+
+  @Test
+  public void testImmutable() {
+    ImmutableHTableDescriptor htd = new ImmutableHTableDescriptor(
+      new HTableDescriptor(TableName.valueOf(name.getMethodName())));
+    TEST_FUNCTION.forEach(f -> {
+      try {
+        f.accept(htd);
+        fail("ImmutableHTableDescriptor can't be modified!!!");
+      } catch (UnsupportedOperationException e) {
+      }
+    });
+  }
+
+  @Test
+  public void testClassMethodsAreBuilderStyle() {
+  /* ImmutableHTableDescriptor should have a builder style setup where setXXX/addXXX methods
+   * can be chainable together:
+   * . For example:
+   * ImmutableHTableDescriptor d
+   *   = new ImmutableHTableDescriptor()
+   *     .setFoo(foo)
+   *     .setBar(bar)
+   *     .setBuz(buz)
+   *
+   * This test ensures that all methods starting with "set" returns the declaring object
+   */
+
+      BuilderStyleTest.assertClassesAreBuilderStyle(ImmutableHTableDescriptor.class);
+  }
+}
\ No newline at end of file


[14/40] hbase git commit: HBASE-17929 addendum fix typo and modify printUsage

Posted by sy...@apache.org.
HBASE-17929 addendum fix typo and modify printUsage


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

Branch: refs/heads/hbase-12439
Commit: 3acd8e4644c111560502ecc06e10d04dd204a06a
Parents: 972e8c8
Author: zhangduo <zh...@apache.org>
Authored: Wed Apr 19 21:02:47 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Apr 19 21:03:09 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/PerformanceEvaluation.java     | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3acd8e46/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index f58c025..a3d3254 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -1907,6 +1907,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println(" replicas        Enable region replica testing. Defaults: 1.");
     System.err.println(" randomSleep     Do a random sleep before each get between 0 and entered value. Defaults: 0");
     System.err.println(" caching         Scan caching to use. Default: 30");
+    System.err.println(" asyncPrefetch   Enable asyncPrefetch for scan");
+    System.err.println(" cacheBlocks     Set the cacheBlocks option for scan. Default: true");
+    System.err.println(" scanReadType    Set the readType option for scan, stream/pread/default. Default: default");
     System.err.println();
     System.err.println(" Note: -D properties will be applied to the conf used. ");
     System.err.println("  For example: ");
@@ -2175,7 +2178,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
       }
 
       final String scanReadType = "--scanReadType=";
-      if (cmd.startsWith(cacheBlocks)) {
+      if (cmd.startsWith(scanReadType)) {
         opts.scanReadType =
             Scan.ReadType.valueOf(cmd.substring(scanReadType.length()).toUpperCase());
         continue;


[24/40] hbase git commit: HBASE-17944 - Removed unused JDK version parsing from ClassSize.

Posted by sy...@apache.org.
HBASE-17944 - Removed unused JDK version parsing from ClassSize.

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

Branch: refs/heads/hbase-12439
Commit: a3b6f4addc7ec90cbebe681e75e4e60f3e6940a5
Parents: 68e48c4
Author: Colm O hEigeartaigh <co...@apache.org>
Authored: Fri Apr 21 09:16:01 2017 +0100
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Apr 21 09:16:23 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/util/ClassSize.java  | 14 --------------
 1 file changed, 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3b6f4ad/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 e1690c0..e064cc0 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
@@ -127,20 +127,6 @@ public class ClassSize {
 
   public static final int STORE_SERVICES;
 
-  /* Are we running on jdk7? */
-  private static final boolean JDK7;
-  static {
-    final String version = System.getProperty("java.version");
-    // Verify String looks like this: 1.6.0_29
-    if (version == null || !version.matches("\\d\\.\\d\\..*")) {
-      throw new RuntimeException("Unexpected version format: " + version);
-    }
-    // Convert char to int
-    int major = (int)(version.charAt(0) - '0');
-    int minor = (int)(version.charAt(2) - '0');
-    JDK7 = major == 1 && minor == 7;
-  }
-
   /**
    * MemoryLayout abstracts details about the JVM object layout. Default implementation is used in
    * case Unsafe is not available.


[20/40] hbase git commit: HBASE-17943 The in-memory flush size is different for each CompactingMemStore located in the same region

Posted by sy...@apache.org.
HBASE-17943 The in-memory flush size is different for each CompactingMemStore located in the same region


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

Branch: refs/heads/hbase-12439
Commit: ea3a27b18df875284899b04fbc5fb58a3120e6c7
Parents: 87f2bb5
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Fri Apr 21 11:20:20 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Apr 21 11:20:20 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/CompactingMemStore.java |  5 +++++
 .../hbase/regionserver/RegionServicesForStores.java   |  2 +-
 .../TestWalAndCompactingMemStoreFlush.java            | 14 +++++++++++---
 3 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ea3a27b1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
index 0c56693..b244997 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
@@ -541,6 +541,11 @@ public class CompactingMemStore extends AbstractMemStore {
     return lowest;
   }
 
+  @VisibleForTesting
+  long getInmemoryFlushSize() {
+    return inmemoryFlushSize;
+  }
+
   // debug method
   public void debug() {
     String msg = "active size=" + this.active.keySize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea3a27b1/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
index ea346ea..8cdfd3b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -82,7 +82,7 @@ public class RegionServicesForStores {
   }
 
   public int getNumStores() {
-    return region.getStores().size();
+    return region.getTableDesc().getColumnFamilyCount();
   }
 
   // methods for tests

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea3a27b1/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
index aae0a4d..2c16399 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
@@ -108,6 +108,12 @@ public class TestWalAndCompactingMemStoreFlush {
     return new Get(row);
   }
 
+  private void verifyInMemoryFlushSize(Region region) {
+    assertEquals(
+      ((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).getInmemoryFlushSize(),
+      ((CompactingMemStore) ((HStore)region.getStore(FAMILY3)).memstore).getInmemoryFlushSize());
+  }
+
   // A helper function to verify edits.
   void verifyEdit(int familyNum, int putNum, Table table) throws IOException {
     Result r = table.get(createGet(familyNum, putNum));
@@ -137,7 +143,7 @@ public class TestWalAndCompactingMemStoreFlush {
 
     // Intialize the region
     Region region = initHRegion("testSelectiveFlushWithEager", conf);
-
+    verifyInMemoryFlushSize(region);
     // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
     for (int i = 1; i <= 1200; i++) {
       region.put(createPut(1, i));        // compacted memstore, all the keys are unique
@@ -378,7 +384,7 @@ public class TestWalAndCompactingMemStoreFlush {
 
     // Initialize the region
     Region region = initHRegion("testSelectiveFlushWithIndexCompaction", conf);
-
+    verifyInMemoryFlushSize(region);
     /*------------------------------------------------------------------------------*/
     /* PHASE I - insertions */
     // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
@@ -635,6 +641,7 @@ public class TestWalAndCompactingMemStoreFlush {
 
     // Intialize the HRegion
     HRegion region = initHRegion("testSelectiveFlushAndWALinDataCompaction", conf);
+    verifyInMemoryFlushSize(region);
     // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
     for (int i = 1; i <= 1200; i++) {
       region.put(createPut(1, i));
@@ -773,6 +780,7 @@ public class TestWalAndCompactingMemStoreFlush {
 
     // Intialize the HRegion
     HRegion region = initHRegion("testSelectiveFlushWithBasicAndMerge", conf);
+    verifyInMemoryFlushSize(region);
     // Add 1200 entries for CF1 (CompactingMemStore), 100 for CF2 (DefaultMemStore) and 50 for CF3
     for (int i = 1; i <= 1200; i++) {
       region.put(createPut(1, i));
@@ -907,7 +915,7 @@ public class TestWalAndCompactingMemStoreFlush {
 
     // Successfully initialize the HRegion
     HRegion region = initHRegion("testSelectiveFlushAndWALinDataCompaction", conf);
-
+    verifyInMemoryFlushSize(region);
     Thread[] threads = new Thread[25];
     for (int i = 0; i < threads.length; i++) {
       int id = i * 10000;


[02/40] hbase git commit: HBASE-17912 - Avoid major compactions on region server startup

Posted by sy...@apache.org.
HBASE-17912 - Avoid major compactions on region server startup

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/5eda5fb9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5eda5fb9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5eda5fb9

Branch: refs/heads/hbase-12439
Commit: 5eda5fb9d7d7fd5ae77d862c2e1666787e72ead0
Parents: 3c32032
Author: gjacoby <gj...@salesforce.com>
Authored: Mon Apr 17 16:08:25 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Apr 17 19:41:19 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/HRegionServer.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5eda5fb9/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..d14571b 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
@@ -1628,7 +1628,9 @@ public class HRegionServer extends HasThread implements
     private final HRegionServer instance;
     private final int majorCompactPriority;
     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
-    private long iteration = 0;
+    //Iteration is 1-based rather than 0-based so we don't check for compaction
+    // immediately upon region server startup
+    private long iteration = 1;
 
     CompactionChecker(final HRegionServer h, final int sleepTime,
         final Stoppable stopper) {