You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/12/31 12:45:37 UTC

[01/47] hbase git commit: HBASE-21570 Add write buffer periodic flush support for AsyncBufferedMutator [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-21512 e7a122780 -> b33b072de (forced update)


HBASE-21570 Add write buffer periodic flush support for AsyncBufferedMutator


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

Branch: refs/heads/HBASE-21512
Commit: b09b87d143730db00ec56114a752d3a74f8982c4
Parents: da9508d
Author: zhangduo <zh...@apache.org>
Authored: Tue Dec 11 08:39:43 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Tue Dec 11 14:51:26 2018 +0800

----------------------------------------------------------------------
 .../hbase/client/AsyncBufferedMutator.java      |  16 +-
 .../client/AsyncBufferedMutatorBuilder.java     |  19 +++
 .../client/AsyncBufferedMutatorBuilderImpl.java |  19 ++-
 .../hbase/client/AsyncBufferedMutatorImpl.java  |  67 +++++---
 .../client/AsyncConnectionConfiguration.java    |  37 +++--
 .../hbase/client/AsyncConnectionImpl.java       |  11 +-
 .../hbase/client/TestAsyncBufferMutator.java    | 161 ++++++++++++++++++-
 7 files changed, 277 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b09b87d1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
index 6fe4b9a..7b21eb5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
@@ -18,13 +18,16 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.Closeable;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
 /**
  * Used to communicate with a single HBase table in batches. Obtain an instance from a
  * {@link AsyncConnection} and call {@link #close()} afterwards.
@@ -52,7 +55,9 @@ public interface AsyncBufferedMutator extends Closeable {
    * part of a batch. Currently only supports {@link Put} and {@link Delete} mutations.
    * @param mutation The data to send.
    */
-  CompletableFuture<Void> mutate(Mutation mutation);
+  default CompletableFuture<Void> mutate(Mutation mutation) {
+    return Iterables.getOnlyElement(mutate(Collections.singletonList(mutation)));
+  }
 
   /**
    * Send some {@link Mutation}s to the table. The mutations will be buffered and sent over the wire
@@ -81,4 +86,11 @@ public interface AsyncBufferedMutator extends Closeable {
    * @return The size of the write buffer in bytes.
    */
   long getWriteBufferSize();
+
+  /**
+   * Returns the periodical flush interval, 0 means disabled.
+   */
+  default long getPeriodicalFlushTimeout(TimeUnit unit) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b09b87d1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java
index 45959bb..c617c8e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java
@@ -46,6 +46,25 @@ public interface AsyncBufferedMutatorBuilder {
   AsyncBufferedMutatorBuilder setRetryPause(long pause, TimeUnit unit);
 
   /**
+   * Set the periodical flush interval. If the data in the buffer has not been flush for a long
+   * time, i.e, reach this timeout limit, we will flush it automatically.
+   * <p/>
+   * Notice that, set the timeout to 0 or a negative value means disable periodical flush, not
+   * 'flush immediately'. If you want to flush immediately then you should not use this class, as it
+   * is designed to be 'buffered'.
+   */
+  default AsyncBufferedMutatorBuilder setWriteBufferPeriodicFlush(long timeout, TimeUnit unit) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  /**
+   * Disable the periodical flush, i.e, set the timeout to 0.
+   */
+  default AsyncBufferedMutatorBuilder disableWriteBufferPeriodicFlush() {
+    return setWriteBufferPeriodicFlush(0, TimeUnit.NANOSECONDS);
+  }
+
+  /**
    * Set the max retry times for an operation. Usually it is the max attempt times minus 1.
    * <p>
    * Operation timeout and max attempt times(or max retry times) are both limitations for retrying,

http://git-wip-us.apache.org/repos/asf/hbase/blob/b09b87d1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
index 227d02b..eb8af17 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
@@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
 
 /**
  * The implementation of {@link AsyncBufferedMutatorBuilder}.
@@ -28,14 +29,20 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 class AsyncBufferedMutatorBuilderImpl implements AsyncBufferedMutatorBuilder {
 
+  private final HashedWheelTimer periodicalFlushTimer;
+
   private final AsyncTableBuilder<?> tableBuilder;
 
   private long writeBufferSize;
 
+  private long periodicFlushTimeoutNs;
+
   public AsyncBufferedMutatorBuilderImpl(AsyncConnectionConfiguration connConf,
-      AsyncTableBuilder<?> tableBuilder) {
+      AsyncTableBuilder<?> tableBuilder, HashedWheelTimer periodicalFlushTimer) {
     this.tableBuilder = tableBuilder;
     this.writeBufferSize = connConf.getWriteBufferSize();
+    this.periodicFlushTimeoutNs = connConf.getWriteBufferPeriodicFlushTimeoutNs();
+    this.periodicalFlushTimer = periodicalFlushTimer;
   }
 
   @Override
@@ -77,8 +84,14 @@ class AsyncBufferedMutatorBuilderImpl implements AsyncBufferedMutatorBuilder {
   }
 
   @Override
-  public AsyncBufferedMutator build() {
-    return new AsyncBufferedMutatorImpl(tableBuilder.build(), writeBufferSize);
+  public AsyncBufferedMutatorBuilder setWriteBufferPeriodicFlush(long timeout, TimeUnit unit) {
+    this.periodicFlushTimeoutNs = unit.toNanos(timeout);
+    return this;
   }
 
+  @Override
+  public AsyncBufferedMutator build() {
+    return new AsyncBufferedMutatorImpl(periodicalFlushTimer, tableBuilder.build(), writeBufferSize,
+      periodicFlushTimeoutNs);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b09b87d1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
index 5a92ace..318c6c9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
@@ -29,16 +30,24 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import org.apache.hbase.thirdparty.io.netty.util.Timeout;
+
 /**
  * The implementation of {@link AsyncBufferedMutator}. Simply wrap an {@link AsyncTable}.
  */
 @InterfaceAudience.Private
 class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
 
+  private final HashedWheelTimer periodicalFlushTimer;
+
   private final AsyncTable<?> table;
 
   private final long writeBufferSize;
 
+  private final long periodicFlushTimeoutNs;
+
   private List<Mutation> mutations = new ArrayList<>();
 
   private List<CompletableFuture<Void>> futures = new ArrayList<>();
@@ -47,9 +56,15 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
 
   private boolean closed;
 
-  AsyncBufferedMutatorImpl(AsyncTable<?> table, long writeBufferSize) {
+  @VisibleForTesting
+  Timeout periodicFlushTask;
+
+  AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table,
+      long writeBufferSize, long periodicFlushTimeoutNs) {
+    this.periodicalFlushTimer = periodicalFlushTimer;
     this.table = table;
     this.writeBufferSize = writeBufferSize;
+    this.periodicFlushTimeoutNs = periodicFlushTimeoutNs;
   }
 
   @Override
@@ -62,7 +77,13 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
     return table.getConfiguration();
   }
 
-  private void internalFlush() {
+  // will be overridden in test
+  @VisibleForTesting
+  protected void internalFlush() {
+    if (periodicFlushTask != null) {
+      periodicFlushTask.cancel();
+      periodicFlushTask = null;
+    }
     List<Mutation> toSend = this.mutations;
     if (toSend.isEmpty()) {
       return;
@@ -86,29 +107,10 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
   }
 
   @Override
-  public CompletableFuture<Void> mutate(Mutation mutation) {
-    CompletableFuture<Void> future = new CompletableFuture<Void>();
-    long heapSize = mutation.heapSize();
-    synchronized (this) {
-      if (closed) {
-        future.completeExceptionally(new IOException("Already closed"));
-        return future;
-      }
-      mutations.add(mutation);
-      futures.add(future);
-      bufferedSize += heapSize;
-      if (bufferedSize >= writeBufferSize) {
-        internalFlush();
-      }
-    }
-    return future;
-  }
-
-  @Override
   public List<CompletableFuture<Void>> mutate(List<? extends Mutation> mutations) {
     List<CompletableFuture<Void>> futures =
-        Stream.<CompletableFuture<Void>> generate(CompletableFuture::new).limit(mutations.size())
-            .collect(Collectors.toList());
+      Stream.<CompletableFuture<Void>> generate(CompletableFuture::new).limit(mutations.size())
+        .collect(Collectors.toList());
     long heapSize = mutations.stream().mapToLong(m -> m.heapSize()).sum();
     synchronized (this) {
       if (closed) {
@@ -116,6 +118,20 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
         futures.forEach(f -> f.completeExceptionally(ioe));
         return futures;
       }
+      if (this.mutations.isEmpty() && periodicFlushTimeoutNs > 0) {
+        periodicFlushTask = periodicalFlushTimer.newTimeout(timeout -> {
+          synchronized (AsyncBufferedMutatorImpl.this) {
+            // confirm that we are still valid, if there is already an internalFlush call before us,
+            // then we should not execute any more. And in internalFlush we will set periodicFlush
+            // to null, and since we may schedule a new one, so here we check whether the references
+            // are equal.
+            if (timeout == periodicFlushTask) {
+              periodicFlushTask = null;
+              internalFlush();
+            }
+          }
+        }, periodicFlushTimeoutNs, TimeUnit.NANOSECONDS);
+      }
       this.mutations.addAll(mutations);
       this.futures.addAll(futures);
       bufferedSize += heapSize;
@@ -141,4 +157,9 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
   public long getWriteBufferSize() {
     return writeBufferSize;
   }
+
+  @Override
+  public long getPeriodicalFlushTimeout(TimeUnit unit) {
+    return unit.convert(periodicFlushTimeoutNs, TimeUnit.NANOSECONDS);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b09b87d1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
index bd2add8..915e9dd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java
@@ -39,11 +39,12 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
 import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hbase.client.AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT;
 import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
+import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS;
+import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.hbase.client.ConnectionConfiguration.WRITE_BUFFER_SIZE_KEY;
 
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -91,32 +92,38 @@ class AsyncConnectionConfiguration {
 
   private final long writeBufferSize;
 
+  private final long writeBufferPeriodicFlushTimeoutNs;
+
   @SuppressWarnings("deprecation")
   AsyncConnectionConfiguration(Configuration conf) {
     this.metaOperationTimeoutNs = TimeUnit.MILLISECONDS.toNanos(
       conf.getLong(HBASE_CLIENT_META_OPERATION_TIMEOUT, DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT));
     this.operationTimeoutNs = TimeUnit.MILLISECONDS.toNanos(
       conf.getLong(HBASE_CLIENT_OPERATION_TIMEOUT, DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT));
-    this.rpcTimeoutNs = TimeUnit.MILLISECONDS
-        .toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY, DEFAULT_HBASE_RPC_TIMEOUT));
+    this.rpcTimeoutNs =
+      TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY, DEFAULT_HBASE_RPC_TIMEOUT));
     this.readRpcTimeoutNs =
-        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_READ_TIMEOUT_KEY, rpcTimeoutNs));
+      TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_READ_TIMEOUT_KEY, rpcTimeoutNs));
     this.writeRpcTimeoutNs =
-        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_WRITE_TIMEOUT_KEY, rpcTimeoutNs));
+      TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_WRITE_TIMEOUT_KEY, rpcTimeoutNs));
     this.pauseNs =
-        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_CLIENT_PAUSE, DEFAULT_HBASE_CLIENT_PAUSE));
+      TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_CLIENT_PAUSE, DEFAULT_HBASE_CLIENT_PAUSE));
     this.maxRetries = conf.getInt(HBASE_CLIENT_RETRIES_NUMBER, DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
     this.startLogErrorsCnt =
-        conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
+      conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
     this.scanTimeoutNs = TimeUnit.MILLISECONDS
-        .toNanos(HBaseConfiguration.getInt(conf, HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
-          HBASE_REGIONSERVER_LEASE_PERIOD_KEY, DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD));
+      .toNanos(HBaseConfiguration.getInt(conf, HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
+        HBASE_REGIONSERVER_LEASE_PERIOD_KEY, DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD));
     this.scannerCaching =
-        conf.getInt(HBASE_CLIENT_SCANNER_CACHING, DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
-    this.metaScannerCaching = conf.getInt(HBASE_META_SCANNER_CACHING, DEFAULT_HBASE_META_SCANNER_CACHING);
+      conf.getInt(HBASE_CLIENT_SCANNER_CACHING, DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
+    this.metaScannerCaching =
+      conf.getInt(HBASE_META_SCANNER_CACHING, DEFAULT_HBASE_META_SCANNER_CACHING);
     this.scannerMaxResultSize = conf.getLong(HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
       DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
-    this.writeBufferSize =  conf.getLong(WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT);
+    this.writeBufferSize = conf.getLong(WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT);
+    this.writeBufferPeriodicFlushTimeoutNs =
+      TimeUnit.MILLISECONDS.toNanos(conf.getLong(WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS,
+        WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT));
   }
 
   long getMetaOperationTimeoutNs() {
@@ -159,7 +166,7 @@ class AsyncConnectionConfiguration {
     return scannerCaching;
   }
 
-  int getMetaScannerCaching(){
+  int getMetaScannerCaching() {
     return metaScannerCaching;
   }
 
@@ -170,4 +177,8 @@ class AsyncConnectionConfiguration {
   long getWriteBufferSize() {
     return writeBufferSize;
   }
+
+  long getWriteBufferPeriodicFlushTimeoutNs() {
+    return writeBufferPeriodicFlushTimeoutNs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b09b87d1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index a05764e..078395b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -68,7 +68,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   @VisibleForTesting
   static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer(
-      Threads.newDaemonThreadFactory("Async-Client-Retry-Timer"), 10, TimeUnit.MILLISECONDS);
+    Threads.newDaemonThreadFactory("Async-Client-Retry-Timer"), 10, TimeUnit.MILLISECONDS);
 
   private static final String RESOLVE_HOSTNAME_ON_FAIL_KEY = "hbase.resolve.hostnames.on.failure";
 
@@ -193,7 +193,7 @@ class AsyncConnectionImpl implements AsyncConnection {
         String msg = "ZooKeeper available but no active master location found";
         LOG.info(msg);
         this.masterStubMakeFuture.getAndSet(null)
-            .completeExceptionally(new MasterNotRunningException(msg));
+          .completeExceptionally(new MasterNotRunningException(msg));
         return;
       }
       try {
@@ -216,7 +216,7 @@ class AsyncConnectionImpl implements AsyncConnection {
           });
       } catch (IOException e) {
         this.masterStubMakeFuture.getAndSet(null)
-            .completeExceptionally(new IOException("Failed to create async master stub", e));
+          .completeExceptionally(new IOException("Failed to create async master stub", e));
       }
     });
   }
@@ -317,12 +317,13 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   @Override
   public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {
-    return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName));
+    return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName), RETRY_TIMER);
   }
 
   @Override
   public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
       ExecutorService pool) {
-    return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool));
+    return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
+      RETRY_TIMER);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b09b87d1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java
index 9fe4ca7..6eed326 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java
@@ -19,7 +19,9 @@ package org.apache.hadoop.hbase.client;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -31,6 +33,7 @@ import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -45,12 +48,15 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+import org.apache.hbase.thirdparty.io.netty.util.Timeout;
+
 @Category({ MediumTests.class, ClientTests.class })
 public class TestAsyncBufferMutator {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAsyncBufferMutator.class);
+    HBaseClassTestRule.forClass(TestAsyncBufferMutator.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
@@ -96,10 +102,10 @@ public class TestAsyncBufferMutator {
   private void test(TableName tableName) throws InterruptedException {
     List<CompletableFuture<Void>> futures = new ArrayList<>();
     try (AsyncBufferedMutator mutator =
-        CONN.getBufferedMutatorBuilder(tableName).setWriteBufferSize(16 * 1024).build()) {
+      CONN.getBufferedMutatorBuilder(tableName).setWriteBufferSize(16 * 1024).build()) {
       List<CompletableFuture<Void>> fs = mutator.mutate(IntStream.range(0, COUNT / 2)
-          .mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
-          .collect(Collectors.toList()));
+        .mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE))
+        .collect(Collectors.toList()));
       // exceeded the write buffer size, a flush will be called directly
       fs.forEach(f -> f.join());
       IntStream.range(COUNT / 2, COUNT).forEach(i -> {
@@ -115,9 +121,9 @@ public class TestAsyncBufferMutator {
     futures.forEach(f -> f.join());
     AsyncTable<?> table = CONN.getTable(tableName);
     IntStream.range(0, COUNT).mapToObj(i -> new Get(Bytes.toBytes(i))).map(g -> table.get(g).join())
-        .forEach(r -> {
-          assertArrayEquals(VALUE, r.getValue(CF, CQ));
-        });
+      .forEach(r -> {
+        assertArrayEquals(VALUE, r.getValue(CF, CQ));
+      });
   }
 
   @Test
@@ -142,4 +148,145 @@ public class TestAsyncBufferMutator {
       }
     }
   }
+
+  @Test
+  public void testNoPeriodicFlush() throws InterruptedException, ExecutionException {
+    try (AsyncBufferedMutator mutator =
+      CONN.getBufferedMutatorBuilder(TABLE_NAME).disableWriteBufferPeriodicFlush().build()) {
+      Put put = new Put(Bytes.toBytes(0)).addColumn(CF, CQ, VALUE);
+      CompletableFuture<?> future = mutator.mutate(put);
+      Thread.sleep(2000);
+      // assert that we have not flushed it out
+      assertFalse(future.isDone());
+      mutator.flush();
+      future.get();
+    }
+    AsyncTable<?> table = CONN.getTable(TABLE_NAME);
+    assertArrayEquals(VALUE, table.get(new Get(Bytes.toBytes(0))).get().getValue(CF, CQ));
+  }
+
+  @Test
+  public void testPeriodicFlush() throws InterruptedException, ExecutionException {
+    AsyncBufferedMutator mutator = CONN.getBufferedMutatorBuilder(TABLE_NAME)
+      .setWriteBufferPeriodicFlush(1, TimeUnit.SECONDS).build();
+    Put put = new Put(Bytes.toBytes(0)).addColumn(CF, CQ, VALUE);
+    CompletableFuture<?> future = mutator.mutate(put);
+    future.get();
+    AsyncTable<?> table = CONN.getTable(TABLE_NAME);
+    assertArrayEquals(VALUE, table.get(new Get(Bytes.toBytes(0))).get().getValue(CF, CQ));
+  }
+
+  // a bit deep into the implementation
+  @Test
+  public void testCancelPeriodicFlush() throws InterruptedException, ExecutionException {
+    Put put = new Put(Bytes.toBytes(0)).addColumn(CF, CQ, VALUE);
+    try (AsyncBufferedMutatorImpl mutator = (AsyncBufferedMutatorImpl) CONN
+      .getBufferedMutatorBuilder(TABLE_NAME).setWriteBufferPeriodicFlush(1, TimeUnit.SECONDS)
+      .setWriteBufferSize(10 * put.heapSize()).build()) {
+      List<CompletableFuture<?>> futures = new ArrayList<>();
+      futures.add(mutator.mutate(put));
+      Timeout task = mutator.periodicFlushTask;
+      // we should have scheduled a periodic flush task
+      assertNotNull(task);
+      for (int i = 1;; i++) {
+        futures.add(mutator.mutate(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, VALUE)));
+        if (mutator.periodicFlushTask == null) {
+          break;
+        }
+      }
+      assertTrue(task.isCancelled());
+      CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join();
+      AsyncTable<?> table = CONN.getTable(TABLE_NAME);
+      for (int i = 0; i < futures.size(); i++) {
+        assertArrayEquals(VALUE, table.get(new Get(Bytes.toBytes(i))).get().getValue(CF, CQ));
+      }
+    }
+  }
+
+  @Test
+  public void testCancelPeriodicFlushByManuallyFlush()
+      throws InterruptedException, ExecutionException {
+    try (AsyncBufferedMutatorImpl mutator =
+      (AsyncBufferedMutatorImpl) CONN.getBufferedMutatorBuilder(TABLE_NAME)
+        .setWriteBufferPeriodicFlush(1, TimeUnit.SECONDS).build()) {
+      CompletableFuture<?> future =
+        mutator.mutate(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, VALUE));
+      Timeout task = mutator.periodicFlushTask;
+      // we should have scheduled a periodic flush task
+      assertNotNull(task);
+      mutator.flush();
+      assertTrue(task.isCancelled());
+      future.get();
+      AsyncTable<?> table = CONN.getTable(TABLE_NAME);
+      assertArrayEquals(VALUE, table.get(new Get(Bytes.toBytes(0))).get().getValue(CF, CQ));
+    }
+  }
+
+  @Test
+  public void testCancelPeriodicFlushByClose() throws InterruptedException, ExecutionException {
+    CompletableFuture<?> future;
+    Timeout task;
+    try (AsyncBufferedMutatorImpl mutator =
+      (AsyncBufferedMutatorImpl) CONN.getBufferedMutatorBuilder(TABLE_NAME)
+        .setWriteBufferPeriodicFlush(1, TimeUnit.SECONDS).build()) {
+      future = mutator.mutate(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, VALUE));
+      task = mutator.periodicFlushTask;
+      // we should have scheduled a periodic flush task
+      assertNotNull(task);
+    }
+    assertTrue(task.isCancelled());
+    future.get();
+    AsyncTable<?> table = CONN.getTable(TABLE_NAME);
+    assertArrayEquals(VALUE, table.get(new Get(Bytes.toBytes(0))).get().getValue(CF, CQ));
+  }
+
+  private static final class AsyncBufferMutatorForTest extends AsyncBufferedMutatorImpl {
+
+    private int flushCount;
+
+    AsyncBufferMutatorForTest(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table,
+        long writeBufferSize, long periodicFlushTimeoutNs) {
+      super(periodicalFlushTimer, table, writeBufferSize, periodicFlushTimeoutNs);
+    }
+
+    @Override
+    protected void internalFlush() {
+      flushCount++;
+      super.internalFlush();
+    }
+  }
+
+  @Test
+  public void testRaceBetweenNormalFlushAndPeriodicFlush()
+      throws InterruptedException, ExecutionException {
+    Put put = new Put(Bytes.toBytes(0)).addColumn(CF, CQ, VALUE);
+    try (AsyncBufferMutatorForTest mutator =
+      new AsyncBufferMutatorForTest(AsyncConnectionImpl.RETRY_TIMER, CONN.getTable(TABLE_NAME),
+        10 * put.heapSize(), TimeUnit.MILLISECONDS.toNanos(200))) {
+      CompletableFuture<?> future = mutator.mutate(put);
+      Timeout task = mutator.periodicFlushTask;
+      // we should have scheduled a periodic flush task
+      assertNotNull(task);
+      synchronized (mutator) {
+        // synchronized on mutator to prevent periodic flush to be executed
+        Thread.sleep(500);
+        // the timeout should be issued
+        assertTrue(task.isExpired());
+        // but no flush is issued as we hold the lock
+        assertEquals(0, mutator.flushCount);
+        assertFalse(future.isDone());
+        // manually flush, then release the lock
+        mutator.flush();
+      }
+      // this is a bit deep into the implementation in netty but anyway let's add a check here to
+      // confirm that an issued timeout can not be canceled by netty framework.
+      assertFalse(task.isCancelled());
+      // and the mutation is done
+      future.get();
+      AsyncTable<?> table = CONN.getTable(TABLE_NAME);
+      assertArrayEquals(VALUE, table.get(new Get(Bytes.toBytes(0))).get().getValue(CF, CQ));
+      // only the manual flush, the periodic flush should have been canceled by us
+      assertEquals(1, mutator.flushCount);
+    }
+  }
 }


[35/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
new file mode 100644
index 0000000..f2c0743
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
@@ -0,0 +1,512 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+public class TTableName implements org.apache.thrift.TBase<TTableName, TTableName._Fields>, java.io.Serializable, Cloneable, Comparable<TTableName> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableName");
+
+  private static final org.apache.thrift.protocol.TField NS_FIELD_DESC = new org.apache.thrift.protocol.TField("ns", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField QUALIFIER_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifier", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TTableNameStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TTableNameTupleSchemeFactory());
+  }
+
+  public ByteBuffer ns; // required
+  public ByteBuffer qualifier; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NS((short)1, "ns"),
+    QUALIFIER((short)2, "qualifier");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NS
+          return NS;
+        case 2: // QUALIFIER
+          return QUALIFIER;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.QUALIFIER, new org.apache.thrift.meta_data.FieldMetaData("qualifier", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTableName.class, metaDataMap);
+  }
+
+  public TTableName() {
+  }
+
+  public TTableName(
+    ByteBuffer ns,
+    ByteBuffer qualifier)
+  {
+    this();
+    this.ns = org.apache.thrift.TBaseHelper.copyBinary(ns);
+    this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(qualifier);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TTableName(TTableName other) {
+    if (other.isSetNs()) {
+      this.ns = org.apache.thrift.TBaseHelper.copyBinary(other.ns);
+    }
+    if (other.isSetQualifier()) {
+      this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(other.qualifier);
+    }
+  }
+
+  public TTableName deepCopy() {
+    return new TTableName(this);
+  }
+
+  @Override
+  public void clear() {
+    this.ns = null;
+    this.qualifier = null;
+  }
+
+  public byte[] getNs() {
+    setNs(org.apache.thrift.TBaseHelper.rightSize(ns));
+    return ns == null ? null : ns.array();
+  }
+
+  public ByteBuffer bufferForNs() {
+    return org.apache.thrift.TBaseHelper.copyBinary(ns);
+  }
+
+  public TTableName setNs(byte[] ns) {
+    this.ns = ns == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(ns, ns.length));
+    return this;
+  }
+
+  public TTableName setNs(ByteBuffer ns) {
+    this.ns = org.apache.thrift.TBaseHelper.copyBinary(ns);
+    return this;
+  }
+
+  public void unsetNs() {
+    this.ns = null;
+  }
+
+  /** Returns true if field ns is set (has been assigned a value) and false otherwise */
+  public boolean isSetNs() {
+    return this.ns != null;
+  }
+
+  public void setNsIsSet(boolean value) {
+    if (!value) {
+      this.ns = null;
+    }
+  }
+
+  public byte[] getQualifier() {
+    setQualifier(org.apache.thrift.TBaseHelper.rightSize(qualifier));
+    return qualifier == null ? null : qualifier.array();
+  }
+
+  public ByteBuffer bufferForQualifier() {
+    return org.apache.thrift.TBaseHelper.copyBinary(qualifier);
+  }
+
+  public TTableName setQualifier(byte[] qualifier) {
+    this.qualifier = qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(qualifier, qualifier.length));
+    return this;
+  }
+
+  public TTableName setQualifier(ByteBuffer qualifier) {
+    this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(qualifier);
+    return this;
+  }
+
+  public void unsetQualifier() {
+    this.qualifier = null;
+  }
+
+  /** Returns true if field qualifier is set (has been assigned a value) and false otherwise */
+  public boolean isSetQualifier() {
+    return this.qualifier != null;
+  }
+
+  public void setQualifierIsSet(boolean value) {
+    if (!value) {
+      this.qualifier = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NS:
+      if (value == null) {
+        unsetNs();
+      } else {
+        setNs((ByteBuffer)value);
+      }
+      break;
+
+    case QUALIFIER:
+      if (value == null) {
+        unsetQualifier();
+      } else {
+        setQualifier((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NS:
+      return getNs();
+
+    case QUALIFIER:
+      return getQualifier();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NS:
+      return isSetNs();
+    case QUALIFIER:
+      return isSetQualifier();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TTableName)
+      return this.equals((TTableName)that);
+    return false;
+  }
+
+  public boolean equals(TTableName that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_ns = true && this.isSetNs();
+    boolean that_present_ns = true && that.isSetNs();
+    if (this_present_ns || that_present_ns) {
+      if (!(this_present_ns && that_present_ns))
+        return false;
+      if (!this.ns.equals(that.ns))
+        return false;
+    }
+
+    boolean this_present_qualifier = true && this.isSetQualifier();
+    boolean that_present_qualifier = true && that.isSetQualifier();
+    if (this_present_qualifier || that_present_qualifier) {
+      if (!(this_present_qualifier && that_present_qualifier))
+        return false;
+      if (!this.qualifier.equals(that.qualifier))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_ns = true && (isSetNs());
+    list.add(present_ns);
+    if (present_ns)
+      list.add(ns);
+
+    boolean present_qualifier = true && (isSetQualifier());
+    list.add(present_qualifier);
+    if (present_qualifier)
+      list.add(qualifier);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TTableName other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetNs()).compareTo(other.isSetNs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ns, other.ns);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(other.isSetQualifier());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetQualifier()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, other.qualifier);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TTableName(");
+    boolean first = true;
+
+    sb.append("ns:");
+    if (this.ns == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.ns, sb);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("qualifier:");
+    if (this.qualifier == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.qualifier, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (ns == null) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'ns' was not present! Struct: " + toString());
+    }
+    if (qualifier == null) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifier' was not present! Struct: " + toString());
+    }
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TTableNameStandardSchemeFactory implements SchemeFactory {
+    public TTableNameStandardScheme getScheme() {
+      return new TTableNameStandardScheme();
+    }
+  }
+
+  private static class TTableNameStandardScheme extends StandardScheme<TTableName> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TTableName struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.ns = iprot.readBinary();
+              struct.setNsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // QUALIFIER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.qualifier = iprot.readBinary();
+              struct.setQualifierIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TTableName struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.ns != null) {
+        oprot.writeFieldBegin(NS_FIELD_DESC);
+        oprot.writeBinary(struct.ns);
+        oprot.writeFieldEnd();
+      }
+      if (struct.qualifier != null) {
+        oprot.writeFieldBegin(QUALIFIER_FIELD_DESC);
+        oprot.writeBinary(struct.qualifier);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TTableNameTupleSchemeFactory implements SchemeFactory {
+    public TTableNameTupleScheme getScheme() {
+      return new TTableNameTupleScheme();
+    }
+  }
+
+  private static class TTableNameTupleScheme extends TupleScheme<TTableName> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TTableName struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeBinary(struct.ns);
+      oprot.writeBinary(struct.qualifier);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TTableName struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.ns = iprot.readBinary();
+      struct.setNsIsSet(true);
+      struct.qualifier = iprot.readBinary();
+      struct.setQualifierIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
index 74c7255..1e1898c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TTimeRange implements org.apache.thrift.TBase<TTimeRange, TTimeRange._Fields>, java.io.Serializable, Cloneable, Comparable<TTimeRange> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
index 79196f4..6383329 100644
--- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
+++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
@@ -48,7 +48,8 @@ struct TColumnValue {
   2: required binary qualifier,
   3: required binary value,
   4: optional i64 timestamp,
-  5: optional binary tags
+  5: optional binary tags,
+  6: optional byte type
 }
 
 /**
@@ -67,6 +68,7 @@ struct TResult {
   1: optional binary row,
   2: required list<TColumnValue> columnValues,
   3: optional bool stale = false
+  4: optional bool partial = false
 }
 
 /**
@@ -90,6 +92,7 @@ enum TDeleteType {
  */
 
 enum TDurability {
+  USE_DEFAULT = 0,
   SKIP_WAL = 1,
   ASYNC_WAL = 2,
   SYNC_WAL = 3,
@@ -139,6 +142,11 @@ struct TGet {
   8: optional TAuthorization authorizations
   9: optional TConsistency consistency
   10: optional i32 targetReplicaId
+  11: optional bool cacheBlocks
+  12: optional i32 storeLimit
+  13: optional i32 storeOffset
+  14: optional bool existence_only
+
 }
 
 /**
@@ -211,6 +219,7 @@ struct TIncrement {
   4: optional map<binary, binary> attributes,
   5: optional TDurability durability
   6: optional TCellVisibility cellVisibility
+  7: optional bool returnResults
 }
 
 /* 
@@ -222,6 +231,7 @@ struct TAppend {
   3: optional map<binary, binary> attributes,
   4: optional TDurability durability
   5: optional TCellVisibility cellVisibility
+  6: optional bool returnResults
 }
 
 enum TReadType {
@@ -258,8 +268,8 @@ struct TScan {
  * Atomic mutation for the specified row. It can be either Put or Delete.
  */
 union TMutation {
-  1: TPut put,
-  2: TDelete deleteSingle,
+  1: TPut put
+  2: TDelete deleteSingle
 }
 
 /**
@@ -305,6 +315,112 @@ enum TCompareOp {
   NO_OP = 6
 }
 
+enum TBloomFilterType {
+/**
+   * Bloomfilters disabled
+   */
+  NONE = 0,
+  /**
+   * Bloom enabled with Table row as Key
+   */
+  ROW = 1,
+  /**
+   * Bloom enabled with Table row &amp; column (family+qualifier) as Key
+   */
+  ROWCOL = 2,
+  /**
+   * Bloom enabled with Table row prefix as Key, specify the length of the prefix
+   */
+  ROWPREFIX_FIXED_LENGTH = 3,
+  /**
+   * Bloom enabled with Table row prefix as Key, specify the delimiter of the prefix
+   */
+  ROWPREFIX_DELIMITED = 4
+}
+
+enum TCompressionAlgorithm {
+  LZO = 0,
+  GZ = 1,
+  NONE = 2,
+  SNAPPY = 3,
+  LZ4 = 4,
+  BZIP2 = 5,
+  ZSTD = 6
+}
+
+enum TDataBlockEncoding {
+/** Disable data block encoding. */
+  NONE = 0,
+  // id 1 is reserved for the BITSET algorithm to be added later
+  PREFIX = 2,
+  DIFF  = 3,
+  FAST_DIFF = 4,
+  // id 5 is reserved for the COPY_KEY algorithm for benchmarking
+  // COPY_KEY(5, "org.apache.hadoop.hbase.io.encoding.CopyKeyDataBlockEncoder"),
+  // PREFIX_TREE(6, "org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec"),
+  ROW_INDEX_V1 = 7
+}
+
+enum TKeepDeletedCells {
+  /** Deleted Cells are not retained. */
+  FALSE = 0,
+  /**
+   * Deleted Cells are retained until they are removed by other means
+   * such TTL or VERSIONS.
+   * If no TTL is specified or no new versions of delete cells are
+   * written, they are retained forever.
+   */
+  TRUE = 1,
+  /**
+   * Deleted Cells are retained until the delete marker expires due to TTL.
+   * This is useful when TTL is combined with MIN_VERSIONS and one
+   * wants to keep a minimum number of versions around but at the same
+   * time remove deleted cells after the TTL.
+   */
+  TTL = 2
+}
+
+struct TTableName {
+  1: required binary ns
+  2: required binary qualifier
+}
+
+struct TColumnFamilyDescriptor {
+  1: required binary name
+  2: optional map<binary, binary> attributes
+  3: optional map<string, string> configuration
+  4: optional i32 blockSize
+  5: optional TBloomFilterType bloomnFilterType
+  6: optional TCompressionAlgorithm compressionType
+  7: optional i16 dfsReplication
+  8: optional TDataBlockEncoding dataBlockEncoding
+  9: optional TKeepDeletedCells keepDeletedCells
+  10: optional i32 maxVersions
+  11: optional i32 minVersions
+  12: optional i32 scope
+  13: optional i32 timeToLive
+  14: optional bool blockCacheEnabled
+  15: optional bool cacheBloomsOnWrite
+  16: optional bool cacheDataOnWrite
+  17: optional bool cacheIndexesOnWrite
+  18: optional bool compressTags
+  19: optional bool evictBlocksOnClose
+  20: optional bool inMemory
+
+}
+
+struct TTableDescriptor {
+ 1: required TTableName tableName
+ 2: optional list<TColumnFamilyDescriptor> columns
+ 3: optional map<binary, binary> attributes
+ 4: optional TDurability durability
+}
+
+struct TNamespaceDescriptor {
+1: required string name
+2: optional map<string, string> configuration
+}
+
 
 //
 // Exceptions
@@ -644,4 +760,111 @@ service THBaseService {
     /** row mutations to execute if the value matches */
     7: required TRowMutations rowMutations
   ) throws (1: TIOError io)
+
+  TTableDescriptor getTableDescriptor(
+  1: required TTableName table
+  ) throws (1: TIOError io)
+
+  list<TTableDescriptor> getTableDescriptors(
+    1: required list<TTableName> tables
+  ) throws (1: TIOError io)
+
+  bool tableExists(
+  1: TTableName tableName
+  ) throws (1: TIOError io)
+
+  list<TTableDescriptor> getTableDescriptorsByPattern(
+    1: required string regex
+    2: required bool includeSysTables
+  ) throws (1: TIOError io)
+
+  list<TTableDescriptor> getTableDescriptorsByNamespace(
+      1: required string name
+  ) throws (1: TIOError io)
+
+  list<TTableName> getTableNamesByPattern(
+    1: required string regex
+    2: required bool includeSysTables
+  ) throws (1: TIOError io)
+
+  list<TTableName> getTableNamesByNamespace(
+    1: required string name
+  ) throws (1: TIOError io)
+
+  void createTable(
+    1: required TTableDescriptor desc
+    2: required list<binary> splitKeys
+  ) throws (1: TIOError io)
+
+  void deleteTable(
+    1: required TTableName tableName
+  ) throws (1: TIOError io)
+
+  void truncateTable(
+    1: required TTableName tableName
+    2: required bool preserveSplits
+  ) throws (1: TIOError io)
+
+  void enableTable(
+    1: required TTableName tableName
+  ) throws (1: TIOError io)
+
+  void disableTable(
+    1: required TTableName tableName
+  ) throws (1: TIOError io)
+
+  bool isTableEnabled(
+    1: required TTableName tableName
+  ) throws (1: TIOError io)
+
+  bool isTableDisabled(
+    1: required TTableName tableName
+  ) throws (1: TIOError io)
+
+  bool isTableAvailable(
+    1: required TTableName tableName
+  ) throws (1: TIOError io)
+
+  bool isTableAvailableWithSplit(
+    1: required TTableName tableName
+    2: required list<binary> splitKeys
+  ) throws (1: TIOError io)
+
+  void addColumnFamily(
+    1: required TTableName tableName
+    2: required TColumnFamilyDescriptor column
+  ) throws (1: TIOError io)
+
+  void deleteColumnFamily(
+    1: required TTableName tableName
+    2: required binary column
+  ) throws (1: TIOError io)
+
+  void modifyColumnFamily(
+    1: required TTableName tableName
+    2: required TColumnFamilyDescriptor column
+  ) throws (1: TIOError io)
+
+  void modifyTable(
+    1: required TTableDescriptor desc
+  ) throws (1: TIOError io)
+
+  void createNamespace(
+    1: required TNamespaceDescriptor namespaceDesc
+  ) throws (1: TIOError io)
+
+  void modifyNamespace(
+    1: required TNamespaceDescriptor namespaceDesc
+  ) throws (1: TIOError io)
+
+  void deleteNamespace(
+    1: required string name
+  ) throws (1: TIOError io)
+
+  TNamespaceDescriptor getNamespaceDescriptor(
+    1: required string name
+  ) throws (1: TIOError io)
+
+  list<TNamespaceDescriptor> listNamespaceDescriptors(
+  ) throws (1: TIOError io)
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
index 1626121..0734f85 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -73,10 +74,12 @@ import org.apache.hadoop.hbase.thrift.ErrorThrowingGetObserver;
 import org.apache.hadoop.hbase.thrift.ThriftMetrics;
 import org.apache.hadoop.hbase.thrift2.generated.TAppend;
 import org.apache.hadoop.hbase.thrift2.generated.TColumn;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
 import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
 import org.apache.hadoop.hbase.thrift2.generated.TConsistency;
+import org.apache.hadoop.hbase.thrift2.generated.TDataBlockEncoding;
 import org.apache.hadoop.hbase.thrift2.generated.TDelete;
 import org.apache.hadoop.hbase.thrift2.generated.TDeleteType;
 import org.apache.hadoop.hbase.thrift2.generated.TDurability;
@@ -86,11 +89,14 @@ import org.apache.hadoop.hbase.thrift2.generated.TIOError;
 import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument;
 import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
 import org.apache.hadoop.hbase.thrift2.generated.TMutation;
+import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TPut;
 import org.apache.hadoop.hbase.thrift2.generated.TReadType;
 import org.apache.hadoop.hbase.thrift2.generated.TResult;
 import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
 import org.apache.hadoop.hbase.thrift2.generated.TScan;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
 import org.apache.hadoop.hbase.thrift2.generated.TTimeRange;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.thrift.TException;
@@ -1592,6 +1598,96 @@ public class TestThriftHBaseServiceHandler {
     assertTrue(tResult.isSetStale());
   }
 
+  @Test
+  public void testDDLOpertions() throws Exception {
+    String namespace = "testDDLOpertionsNamespace";
+    String table = "testDDLOpertionsTable";
+    TTableName tTableName = new TTableName();
+    tTableName.setNs(Bytes.toBytes(namespace));
+    tTableName.setQualifier(Bytes.toBytes(table));
+    ThriftHBaseServiceHandler handler = createHandler();
+    //create name space
+    TNamespaceDescriptor namespaceDescriptor = new TNamespaceDescriptor();
+    namespaceDescriptor.setName(namespace);
+    namespaceDescriptor.putToConfiguration("key1", "value1");
+    namespaceDescriptor.putToConfiguration("key2", "value2");
+    handler.createNamespace(namespaceDescriptor);
+    //list namespace
+    List<TNamespaceDescriptor> namespaceDescriptors = handler.listNamespaceDescriptors();
+    // should have 3 namespace, default hbase and testDDLOpertionsNamespace
+    assertTrue(namespaceDescriptors.size() == 3);
+    //modify namesapce
+    namespaceDescriptor.putToConfiguration("kye3", "value3");
+    handler.modifyNamespace(namespaceDescriptor);
+    //get namespace
+    TNamespaceDescriptor namespaceDescriptorReturned = handler.getNamespaceDescriptor(namespace);
+    assertTrue(namespaceDescriptorReturned.getConfiguration().size() == 3);
+    //create table
+    TTableDescriptor tableDescriptor = new TTableDescriptor();
+    tableDescriptor.setTableName(tTableName);
+    TColumnFamilyDescriptor columnFamilyDescriptor1 = new TColumnFamilyDescriptor();
+    columnFamilyDescriptor1.setName(familyAname);
+    columnFamilyDescriptor1.setDataBlockEncoding(TDataBlockEncoding.DIFF);
+    tableDescriptor.addToColumns(columnFamilyDescriptor1);
+    List<ByteBuffer> splitKeys = new ArrayList<>();
+    splitKeys.add(ByteBuffer.wrap(Bytes.toBytes(5)));
+    handler.createTable(tableDescriptor, splitKeys);
+    //modify table
+    tableDescriptor.setDurability(TDurability.ASYNC_WAL);
+    handler.modifyTable(tableDescriptor);
+    //modify column family
+    columnFamilyDescriptor1.setInMemory(true);
+    handler.modifyColumnFamily(tTableName, columnFamilyDescriptor1);
+    //add column family
+    TColumnFamilyDescriptor columnFamilyDescriptor2 = new TColumnFamilyDescriptor();
+    columnFamilyDescriptor2.setName(familyBname);
+    columnFamilyDescriptor2.setDataBlockEncoding(TDataBlockEncoding.PREFIX);
+    handler.addColumnFamily(tTableName, columnFamilyDescriptor2);
+    //get table descriptor
+    TTableDescriptor tableDescriptorReturned = handler.getTableDescriptor(tTableName);
+    assertTrue(tableDescriptorReturned.getColumns().size() == 2);
+    assertTrue(tableDescriptorReturned.getDurability() ==  TDurability.ASYNC_WAL);
+    TColumnFamilyDescriptor columnFamilyDescriptor1Returned = tableDescriptorReturned.getColumns()
+        .stream().filter(desc -> Bytes.equals(desc.getName(), familyAname)).findFirst().get();
+    assertTrue(columnFamilyDescriptor1Returned.isInMemory() == true);
+    //delete column family
+    handler.deleteColumnFamily(tTableName, ByteBuffer.wrap(familyBname));
+    tableDescriptorReturned = handler.getTableDescriptor(tTableName);
+    assertTrue(tableDescriptorReturned.getColumns().size() == 1);
+    //disable table
+    handler.disableTable(tTableName);
+    assertTrue(handler.isTableDisabled(tTableName));
+    //enable table
+    handler.enableTable(tTableName);
+    assertTrue(handler.isTableEnabled(tTableName));
+    assertTrue(handler.isTableAvailable(tTableName));
+    //truncate table
+    handler.disableTable(tTableName);
+    handler.truncateTable(tTableName, true);
+    assertTrue(handler.isTableAvailable(tTableName));
+    //delete table
+    handler.disableTable(tTableName);
+    handler.deleteTable(tTableName);
+    assertFalse(handler.tableExists(tTableName));
+    //delete namespace
+    handler.deleteNamespace(namespace);
+    namespaceDescriptors = handler.listNamespaceDescriptors();
+    // should have 2 namespace, default and hbase
+    assertTrue(namespaceDescriptors.size() == 2);
+  }
+
+  @Test
+  public void testGetTableDescriptor() throws Exception {
+    ThriftHBaseServiceHandler handler = createHandler();
+    TTableDescriptor tableDescriptor = handler
+        .getTableDescriptor(ThriftUtilities.tableNameFromHBase(TableName.valueOf(tableAname)));
+    TableDescriptor table = ThriftUtilities.tableDescriptorFromThrift(tableDescriptor);
+    assertTrue(table.getTableName().equals(TableName.valueOf(tableAname)));
+    assertTrue(table.getColumnFamilies().length == 2);
+    assertTrue(table.getColumnFamily(familyAname).getMaxVersions() == 3);
+    assertTrue(table.getColumnFamily(familyBname).getMaxVersions() == 2);
+  }
+
   public static class DelayingRegionObserver implements RegionCoprocessor, RegionObserver {
     private static final Logger LOG = LoggerFactory.getLogger(DelayingRegionObserver.class);
     // sleep time in msec


[37/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
index e8f36a0..7388443 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-07-04")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class THBaseService {
 
   public interface Iface {
@@ -282,6 +282,56 @@ public class THBaseService {
      */
     public boolean checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations) throws TIOError, org.apache.thrift.TException;
 
+    public TTableDescriptor getTableDescriptor(TTableName table) throws TIOError, org.apache.thrift.TException;
+
+    public List<TTableDescriptor> getTableDescriptors(List<TTableName> tables) throws TIOError, org.apache.thrift.TException;
+
+    public boolean tableExists(TTableName tableName) throws TIOError, org.apache.thrift.TException;
+
+    public List<TTableDescriptor> getTableDescriptorsByPattern(String regex, boolean includeSysTables) throws TIOError, org.apache.thrift.TException;
+
+    public List<TTableDescriptor> getTableDescriptorsByNamespace(String name) throws TIOError, org.apache.thrift.TException;
+
+    public List<TTableName> getTableNamesByPattern(String regex, boolean includeSysTables) throws TIOError, org.apache.thrift.TException;
+
+    public List<TTableName> getTableNamesByNamespace(String name) throws TIOError, org.apache.thrift.TException;
+
+    public void createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys) throws TIOError, org.apache.thrift.TException;
+
+    public void deleteTable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
+
+    public void truncateTable(TTableName tableName, boolean preserveSplits) throws TIOError, org.apache.thrift.TException;
+
+    public void enableTable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
+
+    public void disableTable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
+
+    public boolean isTableEnabled(TTableName tableName) throws TIOError, org.apache.thrift.TException;
+
+    public boolean isTableDisabled(TTableName tableName) throws TIOError, org.apache.thrift.TException;
+
+    public boolean isTableAvailable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
+
+    public boolean isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys) throws TIOError, org.apache.thrift.TException;
+
+    public void addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws TIOError, org.apache.thrift.TException;
+
+    public void deleteColumnFamily(TTableName tableName, ByteBuffer column) throws TIOError, org.apache.thrift.TException;
+
+    public void modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws TIOError, org.apache.thrift.TException;
+
+    public void modifyTable(TTableDescriptor desc) throws TIOError, org.apache.thrift.TException;
+
+    public void createNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, org.apache.thrift.TException;
+
+    public void modifyNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, org.apache.thrift.TException;
+
+    public void deleteNamespace(String name) throws TIOError, org.apache.thrift.TException;
+
+    public TNamespaceDescriptor getNamespaceDescriptor(String name) throws TIOError, org.apache.thrift.TException;
+
+    public List<TNamespaceDescriptor> listNamespaceDescriptors() throws TIOError, org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -326,6 +376,56 @@ public class THBaseService {
 
     public void checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void getTableDescriptor(TTableName table, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTableDescriptors(List<TTableName> tables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void tableExists(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTableDescriptorsByPattern(String regex, boolean includeSysTables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTableDescriptorsByNamespace(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTableNamesByPattern(String regex, boolean includeSysTables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getTableNamesByNamespace(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void deleteTable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void truncateTable(TTableName tableName, boolean preserveSplits, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void enableTable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void disableTable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void isTableEnabled(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void isTableDisabled(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void isTableAvailable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void deleteColumnFamily(TTableName tableName, ByteBuffer column, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void modifyTable(TTableDescriptor desc, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void createNamespace(TNamespaceDescriptor namespaceDesc, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void modifyNamespace(TNamespaceDescriptor namespaceDesc, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void deleteNamespace(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getNamespaceDescriptor(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void listNamespaceDescriptors(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -892,6 +992,627 @@ public class THBaseService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkAndMutate failed: unknown result");
     }
 
+    public TTableDescriptor getTableDescriptor(TTableName table) throws TIOError, org.apache.thrift.TException
+    {
+      send_getTableDescriptor(table);
+      return recv_getTableDescriptor();
+    }
+
+    public void send_getTableDescriptor(TTableName table) throws org.apache.thrift.TException
+    {
+      getTableDescriptor_args args = new getTableDescriptor_args();
+      args.setTable(table);
+      sendBase("getTableDescriptor", args);
+    }
+
+    public TTableDescriptor recv_getTableDescriptor() throws TIOError, org.apache.thrift.TException
+    {
+      getTableDescriptor_result result = new getTableDescriptor_result();
+      receiveBase(result, "getTableDescriptor");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableDescriptor failed: unknown result");
+    }
+
+    public List<TTableDescriptor> getTableDescriptors(List<TTableName> tables) throws TIOError, org.apache.thrift.TException
+    {
+      send_getTableDescriptors(tables);
+      return recv_getTableDescriptors();
+    }
+
+    public void send_getTableDescriptors(List<TTableName> tables) throws org.apache.thrift.TException
+    {
+      getTableDescriptors_args args = new getTableDescriptors_args();
+      args.setTables(tables);
+      sendBase("getTableDescriptors", args);
+    }
+
+    public List<TTableDescriptor> recv_getTableDescriptors() throws TIOError, org.apache.thrift.TException
+    {
+      getTableDescriptors_result result = new getTableDescriptors_result();
+      receiveBase(result, "getTableDescriptors");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableDescriptors failed: unknown result");
+    }
+
+    public boolean tableExists(TTableName tableName) throws TIOError, org.apache.thrift.TException
+    {
+      send_tableExists(tableName);
+      return recv_tableExists();
+    }
+
+    public void send_tableExists(TTableName tableName) throws org.apache.thrift.TException
+    {
+      tableExists_args args = new tableExists_args();
+      args.setTableName(tableName);
+      sendBase("tableExists", args);
+    }
+
+    public boolean recv_tableExists() throws TIOError, org.apache.thrift.TException
+    {
+      tableExists_result result = new tableExists_result();
+      receiveBase(result, "tableExists");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "tableExists failed: unknown result");
+    }
+
+    public List<TTableDescriptor> getTableDescriptorsByPattern(String regex, boolean includeSysTables) throws TIOError, org.apache.thrift.TException
+    {
+      send_getTableDescriptorsByPattern(regex, includeSysTables);
+      return recv_getTableDescriptorsByPattern();
+    }
+
+    public void send_getTableDescriptorsByPattern(String regex, boolean includeSysTables) throws org.apache.thrift.TException
+    {
+      getTableDescriptorsByPattern_args args = new getTableDescriptorsByPattern_args();
+      args.setRegex(regex);
+      args.setIncludeSysTables(includeSysTables);
+      sendBase("getTableDescriptorsByPattern", args);
+    }
+
+    public List<TTableDescriptor> recv_getTableDescriptorsByPattern() throws TIOError, org.apache.thrift.TException
+    {
+      getTableDescriptorsByPattern_result result = new getTableDescriptorsByPattern_result();
+      receiveBase(result, "getTableDescriptorsByPattern");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableDescriptorsByPattern failed: unknown result");
+    }
+
+    public List<TTableDescriptor> getTableDescriptorsByNamespace(String name) throws TIOError, org.apache.thrift.TException
+    {
+      send_getTableDescriptorsByNamespace(name);
+      return recv_getTableDescriptorsByNamespace();
+    }
+
+    public void send_getTableDescriptorsByNamespace(String name) throws org.apache.thrift.TException
+    {
+      getTableDescriptorsByNamespace_args args = new getTableDescriptorsByNamespace_args();
+      args.setName(name);
+      sendBase("getTableDescriptorsByNamespace", args);
+    }
+
+    public List<TTableDescriptor> recv_getTableDescriptorsByNamespace() throws TIOError, org.apache.thrift.TException
+    {
+      getTableDescriptorsByNamespace_result result = new getTableDescriptorsByNamespace_result();
+      receiveBase(result, "getTableDescriptorsByNamespace");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableDescriptorsByNamespace failed: unknown result");
+    }
+
+    public List<TTableName> getTableNamesByPattern(String regex, boolean includeSysTables) throws TIOError, org.apache.thrift.TException
+    {
+      send_getTableNamesByPattern(regex, includeSysTables);
+      return recv_getTableNamesByPattern();
+    }
+
+    public void send_getTableNamesByPattern(String regex, boolean includeSysTables) throws org.apache.thrift.TException
+    {
+      getTableNamesByPattern_args args = new getTableNamesByPattern_args();
+      args.setRegex(regex);
+      args.setIncludeSysTables(includeSysTables);
+      sendBase("getTableNamesByPattern", args);
+    }
+
+    public List<TTableName> recv_getTableNamesByPattern() throws TIOError, org.apache.thrift.TException
+    {
+      getTableNamesByPattern_result result = new getTableNamesByPattern_result();
+      receiveBase(result, "getTableNamesByPattern");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableNamesByPattern failed: unknown result");
+    }
+
+    public List<TTableName> getTableNamesByNamespace(String name) throws TIOError, org.apache.thrift.TException
+    {
+      send_getTableNamesByNamespace(name);
+      return recv_getTableNamesByNamespace();
+    }
+
+    public void send_getTableNamesByNamespace(String name) throws org.apache.thrift.TException
+    {
+      getTableNamesByNamespace_args args = new getTableNamesByNamespace_args();
+      args.setName(name);
+      sendBase("getTableNamesByNamespace", args);
+    }
+
+    public List<TTableName> recv_getTableNamesByNamespace() throws TIOError, org.apache.thrift.TException
+    {
+      getTableNamesByNamespace_result result = new getTableNamesByNamespace_result();
+      receiveBase(result, "getTableNamesByNamespace");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableNamesByNamespace failed: unknown result");
+    }
+
+    public void createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys) throws TIOError, org.apache.thrift.TException
+    {
+      send_createTable(desc, splitKeys);
+      recv_createTable();
+    }
+
+    public void send_createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys) throws org.apache.thrift.TException
+    {
+      createTable_args args = new createTable_args();
+      args.setDesc(desc);
+      args.setSplitKeys(splitKeys);
+      sendBase("createTable", args);
+    }
+
+    public void recv_createTable() throws TIOError, org.apache.thrift.TException
+    {
+      createTable_result result = new createTable_result();
+      receiveBase(result, "createTable");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void deleteTable(TTableName tableName) throws TIOError, org.apache.thrift.TException
+    {
+      send_deleteTable(tableName);
+      recv_deleteTable();
+    }
+
+    public void send_deleteTable(TTableName tableName) throws org.apache.thrift.TException
+    {
+      deleteTable_args args = new deleteTable_args();
+      args.setTableName(tableName);
+      sendBase("deleteTable", args);
+    }
+
+    public void recv_deleteTable() throws TIOError, org.apache.thrift.TException
+    {
+      deleteTable_result result = new deleteTable_result();
+      receiveBase(result, "deleteTable");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void truncateTable(TTableName tableName, boolean preserveSplits) throws TIOError, org.apache.thrift.TException
+    {
+      send_truncateTable(tableName, preserveSplits);
+      recv_truncateTable();
+    }
+
+    public void send_truncateTable(TTableName tableName, boolean preserveSplits) throws org.apache.thrift.TException
+    {
+      truncateTable_args args = new truncateTable_args();
+      args.setTableName(tableName);
+      args.setPreserveSplits(preserveSplits);
+      sendBase("truncateTable", args);
+    }
+
+    public void recv_truncateTable() throws TIOError, org.apache.thrift.TException
+    {
+      truncateTable_result result = new truncateTable_result();
+      receiveBase(result, "truncateTable");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void enableTable(TTableName tableName) throws TIOError, org.apache.thrift.TException
+    {
+      send_enableTable(tableName);
+      recv_enableTable();
+    }
+
+    public void send_enableTable(TTableName tableName) throws org.apache.thrift.TException
+    {
+      enableTable_args args = new enableTable_args();
+      args.setTableName(tableName);
+      sendBase("enableTable", args);
+    }
+
+    public void recv_enableTable() throws TIOError, org.apache.thrift.TException
+    {
+      enableTable_result result = new enableTable_result();
+      receiveBase(result, "enableTable");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void disableTable(TTableName tableName) throws TIOError, org.apache.thrift.TException
+    {
+      send_disableTable(tableName);
+      recv_disableTable();
+    }
+
+    public void send_disableTable(TTableName tableName) throws org.apache.thrift.TException
+    {
+      disableTable_args args = new disableTable_args();
+      args.setTableName(tableName);
+      sendBase("disableTable", args);
+    }
+
+    public void recv_disableTable() throws TIOError, org.apache.thrift.TException
+    {
+      disableTable_result result = new disableTable_result();
+      receiveBase(result, "disableTable");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public boolean isTableEnabled(TTableName tableName) throws TIOError, org.apache.thrift.TException
+    {
+      send_isTableEnabled(tableName);
+      return recv_isTableEnabled();
+    }
+
+    public void send_isTableEnabled(TTableName tableName) throws org.apache.thrift.TException
+    {
+      isTableEnabled_args args = new isTableEnabled_args();
+      args.setTableName(tableName);
+      sendBase("isTableEnabled", args);
+    }
+
+    public boolean recv_isTableEnabled() throws TIOError, org.apache.thrift.TException
+    {
+      isTableEnabled_result result = new isTableEnabled_result();
+      receiveBase(result, "isTableEnabled");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "isTableEnabled failed: unknown result");
+    }
+
+    public boolean isTableDisabled(TTableName tableName) throws TIOError, org.apache.thrift.TException
+    {
+      send_isTableDisabled(tableName);
+      return recv_isTableDisabled();
+    }
+
+    public void send_isTableDisabled(TTableName tableName) throws org.apache.thrift.TException
+    {
+      isTableDisabled_args args = new isTableDisabled_args();
+      args.setTableName(tableName);
+      sendBase("isTableDisabled", args);
+    }
+
+    public boolean recv_isTableDisabled() throws TIOError, org.apache.thrift.TException
+    {
+      isTableDisabled_result result = new isTableDisabled_result();
+      receiveBase(result, "isTableDisabled");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "isTableDisabled failed: unknown result");
+    }
+
+    public boolean isTableAvailable(TTableName tableName) throws TIOError, org.apache.thrift.TException
+    {
+      send_isTableAvailable(tableName);
+      return recv_isTableAvailable();
+    }
+
+    public void send_isTableAvailable(TTableName tableName) throws org.apache.thrift.TException
+    {
+      isTableAvailable_args args = new isTableAvailable_args();
+      args.setTableName(tableName);
+      sendBase("isTableAvailable", args);
+    }
+
+    public boolean recv_isTableAvailable() throws TIOError, org.apache.thrift.TException
+    {
+      isTableAvailable_result result = new isTableAvailable_result();
+      receiveBase(result, "isTableAvailable");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "isTableAvailable failed: unknown result");
+    }
+
+    public boolean isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys) throws TIOError, org.apache.thrift.TException
+    {
+      send_isTableAvailableWithSplit(tableName, splitKeys);
+      return recv_isTableAvailableWithSplit();
+    }
+
+    public void send_isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys) throws org.apache.thrift.TException
+    {
+      isTableAvailableWithSplit_args args = new isTableAvailableWithSplit_args();
+      args.setTableName(tableName);
+      args.setSplitKeys(splitKeys);
+      sendBase("isTableAvailableWithSplit", args);
+    }
+
+    public boolean recv_isTableAvailableWithSplit() throws TIOError, org.apache.thrift.TException
+    {
+      isTableAvailableWithSplit_result result = new isTableAvailableWithSplit_result();
+      receiveBase(result, "isTableAvailableWithSplit");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "isTableAvailableWithSplit failed: unknown result");
+    }
+
+    public void addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws TIOError, org.apache.thrift.TException
+    {
+      send_addColumnFamily(tableName, column);
+      recv_addColumnFamily();
+    }
+
+    public void send_addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws org.apache.thrift.TException
+    {
+      addColumnFamily_args args = new addColumnFamily_args();
+      args.setTableName(tableName);
+      args.setColumn(column);
+      sendBase("addColumnFamily", args);
+    }
+
+    public void recv_addColumnFamily() throws TIOError, org.apache.thrift.TException
+    {
+      addColumnFamily_result result = new addColumnFamily_result();
+      receiveBase(result, "addColumnFamily");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void deleteColumnFamily(TTableName tableName, ByteBuffer column) throws TIOError, org.apache.thrift.TException
+    {
+      send_deleteColumnFamily(tableName, column);
+      recv_deleteColumnFamily();
+    }
+
+    public void send_deleteColumnFamily(TTableName tableName, ByteBuffer column) throws org.apache.thrift.TException
+    {
+      deleteColumnFamily_args args = new deleteColumnFamily_args();
+      args.setTableName(tableName);
+      args.setColumn(column);
+      sendBase("deleteColumnFamily", args);
+    }
+
+    public void recv_deleteColumnFamily() throws TIOError, org.apache.thrift.TException
+    {
+      deleteColumnFamily_result result = new deleteColumnFamily_result();
+      receiveBase(result, "deleteColumnFamily");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws TIOError, org.apache.thrift.TException
+    {
+      send_modifyColumnFamily(tableName, column);
+      recv_modifyColumnFamily();
+    }
+
+    public void send_modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws org.apache.thrift.TException
+    {
+      modifyColumnFamily_args args = new modifyColumnFamily_args();
+      args.setTableName(tableName);
+      args.setColumn(column);
+      sendBase("modifyColumnFamily", args);
+    }
+
+    public void recv_modifyColumnFamily() throws TIOError, org.apache.thrift.TException
+    {
+      modifyColumnFamily_result result = new modifyColumnFamily_result();
+      receiveBase(result, "modifyColumnFamily");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void modifyTable(TTableDescriptor desc) throws TIOError, org.apache.thrift.TException
+    {
+      send_modifyTable(desc);
+      recv_modifyTable();
+    }
+
+    public void send_modifyTable(TTableDescriptor desc) throws org.apache.thrift.TException
+    {
+      modifyTable_args args = new modifyTable_args();
+      args.setDesc(desc);
+      sendBase("modifyTable", args);
+    }
+
+    public void recv_modifyTable() throws TIOError, org.apache.thrift.TException
+    {
+      modifyTable_result result = new modifyTable_result();
+      receiveBase(result, "modifyTable");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void createNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, org.apache.thrift.TException
+    {
+      send_createNamespace(namespaceDesc);
+      recv_createNamespace();
+    }
+
+    public void send_createNamespace(TNamespaceDescriptor namespaceDesc) throws org.apache.thrift.TException
+    {
+      createNamespace_args args = new createNamespace_args();
+      args.setNamespaceDesc(namespaceDesc);
+      sendBase("createNamespace", args);
+    }
+
+    public void recv_createNamespace() throws TIOError, org.apache.thrift.TException
+    {
+      createNamespace_result result = new createNamespace_result();
+      receiveBase(result, "createNamespace");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void modifyNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, org.apache.thrift.TException
+    {
+      send_modifyNamespace(namespaceDesc);
+      recv_modifyNamespace();
+    }
+
+    public void send_modifyNamespace(TNamespaceDescriptor namespaceDesc) throws org.apache.thrift.TException
+    {
+      modifyNamespace_args args = new modifyNamespace_args();
+      args.setNamespaceDesc(namespaceDesc);
+      sendBase("modifyNamespace", args);
+    }
+
+    public void recv_modifyNamespace() throws TIOError, org.apache.thrift.TException
+    {
+      modifyNamespace_result result = new modifyNamespace_result();
+      receiveBase(result, "modifyNamespace");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void deleteNamespace(String name) throws TIOError, org.apache.thrift.TException
+    {
+      send_deleteNamespace(name);
+      recv_deleteNamespace();
+    }
+
+    public void send_deleteNamespace(String name) throws org.apache.thrift.TException
+    {
+      deleteNamespace_args args = new deleteNamespace_args();
+      args.setName(name);
+      sendBase("deleteNamespace", args);
+    }
+
+    public void recv_deleteNamespace() throws TIOError, org.apache.thrift.TException
+    {
+      deleteNamespace_result result = new deleteNamespace_result();
+      receiveBase(result, "deleteNamespace");
+      if (result.io != null) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public TNamespaceDescriptor getNamespaceDescriptor(String name) throws TIOError, org.apache.thrift.TException
+    {
+      send_getNamespaceDescriptor(name);
+      return recv_getNamespaceDescriptor();
+    }
+
+    public void send_getNamespaceDescriptor(String name) throws org.apache.thrift.TException
+    {
+      getNamespaceDescriptor_args args = new getNamespaceDescriptor_args();
+      args.setName(name);
+      sendBase("getNamespaceDescriptor", args);
+    }
+
+    public TNamespaceDescriptor recv_getNamespaceDescriptor() throws TIOError, org.apache.thrift.TException
+    {
+      getNamespaceDescriptor_result result = new getNamespaceDescriptor_result();
+      receiveBase(result, "getNamespaceDescriptor");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNamespaceDescriptor failed: unknown result");
+    }
+
+    public List<TNamespaceDescriptor> listNamespaceDescriptors() throws TIOError, org.apache.thrift.TException
+    {
+      send_listNamespaceDescriptors();
+      return recv_listNamespaceDescriptors();
+    }
+
+    public void send_listNamespaceDescriptors() throws org.apache.thrift.TException
+    {
+      listNamespaceDescriptors_args args = new listNamespaceDescriptors_args();
+      sendBase("listNamespaceDescriptors", args);
+    }
+
+    public List<TNamespaceDescriptor> recv_listNamespaceDescriptors() throws TIOError, org.apache.thrift.TException
+    {
+      listNamespaceDescriptors_result result = new listNamespaceDescriptors_result();
+      receiveBase(result, "listNamespaceDescriptors");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "listNamespaceDescriptors failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -1649,544 +2370,840 @@ public class THBaseService {
       }
     }
 
-  }
-
-  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(I iface) {
-      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    public void getTableDescriptor(TTableName table, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTableDescriptor_call method_call = new getTableDescriptor_call(table, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
     }
 
-    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
+    public static class getTableDescriptor_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName table;
+      public getTableDescriptor_call(TTableName table, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.table = table;
+      }
 
-    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      processMap.put("exists", new exists());
-      processMap.put("existsAll", new existsAll());
-      processMap.put("get", new get());
-      processMap.put("getMultiple", new getMultiple());
-      processMap.put("put", new put());
-      processMap.put("checkAndPut", new checkAndPut());
-      processMap.put("putMultiple", new putMultiple());
-      processMap.put("deleteSingle", new deleteSingle());
-      processMap.put("deleteMultiple", new deleteMultiple());
-      processMap.put("checkAndDelete", new checkAndDelete());
-      processMap.put("increment", new increment());
-      processMap.put("append", new append());
-      processMap.put("openScanner", new openScanner());
-      processMap.put("getScannerRows", new getScannerRows());
-      processMap.put("closeScanner", new closeScanner());
-      processMap.put("mutateRow", new mutateRow());
-      processMap.put("getScannerResults", new getScannerResults());
-      processMap.put("getRegionLocation", new getRegionLocation());
-      processMap.put("getAllRegionLocations", new getAllRegionLocations());
-      processMap.put("checkAndMutate", new checkAndMutate());
-      return processMap;
-    }
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableDescriptor", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTableDescriptor_args args = new getTableDescriptor_args();
+        args.setTable(table);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
 
-    public static class exists<I extends Iface> extends org.apache.thrift.ProcessFunction<I, exists_args> {
-      public exists() {
-        super("exists");
+      public TTableDescriptor getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTableDescriptor();
       }
+    }
 
-      public exists_args getEmptyArgsInstance() {
-        return new exists_args();
+    public void getTableDescriptors(List<TTableName> tables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTableDescriptors_call method_call = new getTableDescriptors_call(tables, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getTableDescriptors_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private List<TTableName> tables;
+      public getTableDescriptors_call(List<TTableName> tables, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tables = tables;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableDescriptors", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTableDescriptors_args args = new getTableDescriptors_args();
+        args.setTables(tables);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public exists_result getResult(I iface, exists_args args) throws org.apache.thrift.TException {
-        exists_result result = new exists_result();
-        try {
-          result.success = iface.exists(args.table, args.tget);
-          result.setSuccessIsSet(true);
-        } catch (TIOError io) {
-          result.io = io;
+      public List<TTableDescriptor> getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTableDescriptors();
       }
     }
 
-    public static class existsAll<I extends Iface> extends org.apache.thrift.ProcessFunction<I, existsAll_args> {
-      public existsAll() {
-        super("existsAll");
-      }
+    public void tableExists(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      tableExists_call method_call = new tableExists_call(tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public existsAll_args getEmptyArgsInstance() {
-        return new existsAll_args();
+    public static class tableExists_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      public tableExists_call(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("tableExists", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        tableExists_args args = new tableExists_args();
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public existsAll_result getResult(I iface, existsAll_args args) throws org.apache.thrift.TException {
-        existsAll_result result = new existsAll_result();
-        try {
-          result.success = iface.existsAll(args.table, args.tgets);
-        } catch (TIOError io) {
-          result.io = io;
+      public boolean getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_tableExists();
       }
     }
 
-    public static class get<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_args> {
-      public get() {
-        super("get");
-      }
+    public void getTableDescriptorsByPattern(String regex, boolean includeSysTables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTableDescriptorsByPattern_call method_call = new getTableDescriptorsByPattern_call(regex, includeSysTables, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public get_args getEmptyArgsInstance() {
-        return new get_args();
+    public static class getTableDescriptorsByPattern_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String regex;
+      private boolean includeSysTables;
+      public getTableDescriptorsByPattern_call(String regex, boolean includeSysTables, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.regex = regex;
+        this.includeSysTables = includeSysTables;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableDescriptorsByPattern", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTableDescriptorsByPattern_args args = new getTableDescriptorsByPattern_args();
+        args.setRegex(regex);
+        args.setIncludeSysTables(includeSysTables);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public get_result getResult(I iface, get_args args) throws org.apache.thrift.TException {
-        get_result result = new get_result();
-        try {
-          result.success = iface.get(args.table, args.tget);
-        } catch (TIOError io) {
-          result.io = io;
+      public List<TTableDescriptor> getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTableDescriptorsByPattern();
       }
     }
 
-    public static class getMultiple<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getMultiple_args> {
-      public getMultiple() {
-        super("getMultiple");
-      }
+    public void getTableDescriptorsByNamespace(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTableDescriptorsByNamespace_call method_call = new getTableDescriptorsByNamespace_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public getMultiple_args getEmptyArgsInstance() {
-        return new getMultiple_args();
+    public static class getTableDescriptorsByNamespace_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      public getTableDescriptorsByNamespace_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableDescriptorsByNamespace", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTableDescriptorsByNamespace_args args = new getTableDescriptorsByNamespace_args();
+        args.setName(name);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public getMultiple_result getResult(I iface, getMultiple_args args) throws org.apache.thrift.TException {
-        getMultiple_result result = new getMultiple_result();
-        try {
-          result.success = iface.getMultiple(args.table, args.tgets);
-        } catch (TIOError io) {
-          result.io = io;
+      public List<TTableDescriptor> getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTableDescriptorsByNamespace();
       }
     }
 
-    public static class put<I extends Iface> extends org.apache.thrift.ProcessFunction<I, put_args> {
-      public put() {
-        super("put");
-      }
+    public void getTableNamesByPattern(String regex, boolean includeSysTables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTableNamesByPattern_call method_call = new getTableNamesByPattern_call(regex, includeSysTables, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public put_args getEmptyArgsInstance() {
-        return new put_args();
+    public static class getTableNamesByPattern_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String regex;
+      private boolean includeSysTables;
+      public getTableNamesByPattern_call(String regex, boolean includeSysTables, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.regex = regex;
+        this.includeSysTables = includeSysTables;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableNamesByPattern", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTableNamesByPattern_args args = new getTableNamesByPattern_args();
+        args.setRegex(regex);
+        args.setIncludeSysTables(includeSysTables);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public put_result getResult(I iface, put_args args) throws org.apache.thrift.TException {
-        put_result result = new put_result();
-        try {
-          iface.put(args.table, args.tput);
-        } catch (TIOError io) {
-          result.io = io;
+      public List<TTableName> getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTableNamesByPattern();
       }
     }
 
-    public static class checkAndPut<I extends Iface> extends org.apache.thrift.ProcessFunction<I, checkAndPut_args> {
-      public checkAndPut() {
-        super("checkAndPut");
-      }
+    public void getTableNamesByNamespace(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTableNamesByNamespace_call method_call = new getTableNamesByNamespace_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public checkAndPut_args getEmptyArgsInstance() {
-        return new checkAndPut_args();
+    public static class getTableNamesByNamespace_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      public getTableNamesByNamespace_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableNamesByNamespace", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTableNamesByNamespace_args args = new getTableNamesByNamespace_args();
+        args.setName(name);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public checkAndPut_result getResult(I iface, checkAndPut_args args) throws org.apache.thrift.TException {
-        checkAndPut_result result = new checkAndPut_result();
-        try {
-          result.success = iface.checkAndPut(args.table, args.row, args.family, args.qualifier, args.value, args.tput);
-          result.setSuccessIsSet(true);
-        } catch (TIOError io) {
-          result.io = io;
+      public List<TTableName> getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTableNamesByNamespace();
       }
     }
 
-    public static class putMultiple<I extends Iface> extends org.apache.thrift.ProcessFunction<I, putMultiple_args> {
-      public putMultiple() {
-        super("putMultiple");
-      }
+    public void createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      createTable_call method_call = new createTable_call(desc, splitKeys, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public putMultiple_args getEmptyArgsInstance() {
-        return new putMultiple_args();
+    public static class createTable_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableDescriptor desc;
+      private List<ByteBuffer> splitKeys;
+      public createTable_call(TTableDescriptor desc, List<ByteBuffer> splitKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.desc = desc;
+        this.splitKeys = splitKeys;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("createTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        createTable_args args = new createTable_args();
+        args.setDesc(desc);
+        args.setSplitKeys(splitKeys);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public putMultiple_result getResult(I iface, putMultiple_args args) throws org.apache.thrift.TException {
-        putMultiple_result result = new putMultiple_result();
-        try {
-          iface.putMultiple(args.table, args.tputs);
-        } catch (TIOError io) {
-          result.io = io;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_createTable();
       }
     }
 
-    public static class deleteSingle<I extends Iface> extends org.apache.thrift.ProcessFunction<I, deleteSingle_args> {
-      public deleteSingle() {
-        super("deleteSingle");
-      }
+    public void deleteTable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      deleteTable_call method_call = new deleteTable_call(tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public deleteSingle_args getEmptyArgsInstance() {
-        return new deleteSingle_args();
+    public static class deleteTable_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      public deleteTable_call(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("deleteTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        deleteTable_args args = new deleteTable_args();
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public deleteSingle_result getResult(I iface, deleteSingle_args args) throws org.apache.thrift.TException {
-        deleteSingle_result result = new deleteSingle_result();
-        try {
-          iface.deleteSingle(args.table, args.tdelete);
-        } catch (TIOError io) {
-          result.io = io;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_deleteTable();
       }
     }
 
-    public static class deleteMultiple<I extends Iface> extends org.apache.thrift.ProcessFunction<I, deleteMultiple_args> {
-      public deleteMultiple() {
-        super("deleteMultiple");
-      }
+    public void truncateTable(TTableName tableName, boolean preserveSplits, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      truncateTable_call method_call = new truncateTable_call(tableName, preserveSplits, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public deleteMultiple_args getEmptyArgsInstance() {
-        return new deleteMultiple_args();
+    public static class truncateTable_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      private boolean preserveSplits;
+      public truncateTable_call(TTableName tableName, boolean preserveSplits, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
+        this.preserveSplits = preserveSplits;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("truncateTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        truncateTable_args args = new truncateTable_args();
+        args.setTableName(tableName);
+        args.setPreserveSplits(preserveSplits);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public deleteMultiple_result getResult(I iface, deleteMultiple_args args) throws org.apache.thrift.TException {
-        deleteMultiple_result result = new deleteMultiple_result();
-        try {
-          result.success = iface.deleteMultiple(args.table, args.tdeletes);
-        } catch (TIOError io) {
-          result.io = io;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_truncateTable();
       }
     }
 
-    public static class checkAndDelete<I extends Iface> extends org.apache.thrift.ProcessFunction<I, checkAndDelete_args> {
-      public checkAndDelete() {
-        super("checkAndDelete");
-      }
+    public void enableTable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      enableTable_call method_call = new enableTable_call(tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public checkAndDelete_args getEmptyArgsInstance() {
-        return new checkAndDelete_args();
+    public static class enableTable_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      public enableTable_call(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("enableTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        enableTable_args args = new enableTable_args();
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public checkAndDelete_result getResult(I iface, checkAndDelete_args args) throws org.apache.thrift.TException {
-        checkAndDelete_result result = new checkAndDelete_result();
-        try {
-          result.success = iface.checkAndDelete(args.table, args.row, args.family, args.qualifier, args.value, args.tdelete);
-          result.setSuccessIsSet(true);
-        } catch (TIOError io) {
-          result.io = io;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_enableTable();
       }
     }
 
-    public static class increment<I extends Iface> extends org.apache.thrift.ProcessFunction<I, increment_args> {
-      public increment() {
-        super("increment");
-      }
+    public void disableTable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      disableTable_call method_call = new disableTable_call(tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public increment_args getEmptyArgsInstance() {
-        return new increment_args();
+    public static class disableTable_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      public disableTable_call(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("disableTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        disableTable_args args = new disableTable_args();
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public increment_result getResult(I iface, increment_args args) throws org.apache.thrift.TException {
-        increment_result result = new increment_result();
-        try {
-          result.success = iface.increment(args.table, args.tincrement);
-        } catch (TIOError io) {
-          result.io = io;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_disableTable();
       }
     }
 
-    public static class append<I extends Iface> extends org.apache.thrift.ProcessFunction<I, append_args> {
-      public append() {
-        super("append");
-      }
+    public void isTableEnabled(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      isTableEnabled_call method_call = new isTableEnabled_call(tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public append_args getEmptyArgsInstance() {
-        return new append_args();
+    public static class isTableEnabled_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      public isTableEnabled_call(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("isTableEnabled", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        isTableEnabled_args args = new isTableEnabled_args();
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public append_result getResult(I iface, append_args args) throws org.apache.thrift.TException {
-        append_result result = new append_result();
-        try {
-          result.success = iface.append(args.table, args.tappend);
-        } catch (TIOError io) {
-          result.io = io;
+      public boolean getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_isTableEnabled();
       }
     }
 
-    public static class openScanner<I extends Iface> extends org.apache.thrift.ProcessFunction<I, openScanner_args> {
-      public openScanner() {
-        super("openScanner");
-      }
+    public void isTableDisabled(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      isTableDisabled_call method_call = new isTableDisabled_call(tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public openScanner_args getEmptyArgsInstance() {
-        return new openScanner_args();
+    public static class isTableDisabled_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      public isTableDisabled_call(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("isTableDisabled", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        isTableDisabled_args args = new isTableDisabled_args();
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public openScanner_result getResult(I iface, openScanner_args args) throws org.apache.thrift.TException {
-        openScanner_result result = new openScanner_result();
-        try {
-          result.success = iface.openScanner(args.table, args.tscan);
-          result.setSuccessIsSet(true);
-        } catch (TIOError io) {
-          result.io = io;
+      public boolean getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_isTableDisabled();
       }
     }
 
-    public static class getScannerRows<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getScannerRows_args> {
-      public getScannerRows() {
-        super("getScannerRows");
-      }
+    public void isTableAvailable(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      isTableAvailable_call method_call = new isTableAvailable_call(tableName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public getScannerRows_args getEmptyArgsInstance() {
-        return new getScannerRows_args();
+    public static class isTableAvailable_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      public isTableAvailable_call(TTableName tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("isTableAvailable", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        isTableAvailable_args args = new isTableAvailable_args();
+        args.setTableName(tableName);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public getScannerRows_result getResult(I iface, getScannerRows_args args) throws org.apache.thrift.TException {
-        getScannerRows_result result = new getScannerRows_result();
-        try {
-          result.success = iface.getScannerRows(args.scannerId, args.numRows);
-        } catch (TIOError io) {
-          result.io = io;
-        } catch (TIllegalArgument ia) {
-          result.ia = ia;
+      public boolean getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_isTableAvailable();
       }
     }
 
-    public static class closeScanner<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeScanner_args> {
-      public closeScanner() {
-        super("closeScanner");
-      }
+    public void isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      isTableAvailableWithSplit_call method_call = new isTableAvailableWithSplit_call(tableName, splitKeys, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public closeScanner_args getEmptyArgsInstance() {
-        return new closeScanner_args();
+    public static class isTableAvailableWithSplit_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      private List<ByteBuffer> splitKeys;
+      public isTableAvailableWithSplit_call(TTableName tableName, List<ByteBuffer> splitKeys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
+        this.splitKeys = splitKeys;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("isTableAvailableWithSplit", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        isTableAvailableWithSplit_args args = new isTableAvailableWithSplit_args();
+        args.setTableName(tableName);
+        args.setSplitKeys(splitKeys);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public closeScanner_result getResult(I iface, closeScanner_args args) throws org.apache.thrift.TException {
-        closeScanner_result result = new closeScanner_result();
-        try {
-          iface.closeScanner(args.scannerId);
-        } catch (TIOError io) {
-          result.io = io;
-        } catch (TIllegalArgument ia) {
-          result.ia = ia;
+      public boolean getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_isTableAvailableWithSplit();
       }
     }
 
-    public static class mutateRow<I extends Iface> extends org.apache.thrift.ProcessFunction<I, mutateRow_args> {
-      public mutateRow() {
-        super("mutateRow");
-      }
+    public void addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      addColumnFamily_call method_call = new addColumnFamily_call(tableName, column, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public mutateRow_args getEmptyArgsInstance() {
-        return new mutateRow_args();
+    public static class addColumnFamily_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      private TColumnFamilyDescriptor column;
+      public addColumnFamily_call(TTableName tableName, TColumnFamilyDescriptor column, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
+        this.column = column;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("addColumnFamily", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        addColumnFamily_args args = new addColumnFamily_args();
+        args.setTableName(tableName);
+        args.setColumn(column);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public mutateRow_result getResult(I iface, mutateRow_args args) throws org.apache.thrift.TException {
-        mutateRow_result result = new mutateRow_result();
-        try {
-          iface.mutateRow(args.table, args.trowMutations);
-        } catch (TIOError io) {
-          result.io = io;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_addColumnFamily();
       }
     }
 
-    public static class getScannerResults<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getScannerResults_args> {
-      public getScannerResults() {
-        super("getScannerResults");
-      }
+    public void deleteColumnFamily(TTableName tableName, ByteBuffer column, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      deleteColumnFamily_call method_call = new deleteColumnFamily_call(tableName, column, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public getScannerResults_args getEmptyArgsInstance() {
-        return new getScannerResults_args();
+    public static class deleteColumnFamily_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      private ByteBuffer column;
+      public deleteColumnFamily_call(TTableName tableName, ByteBuffer column, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
+        this.column = column;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("deleteColumnFamily", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        deleteColumnFamily_args args = new deleteColumnFamily_args();
+        args.setTableName(tableName);
+        args.setColumn(column);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public getScannerResults_result getResult(I iface, getScannerResults_args args) throws org.apache.thrift.TException {
-        getScannerResults_result result = new getScannerResults_result();
-        try {
-          result.success = iface.getScannerResults(args.table, args.tscan, args.numRows);
-        } catch (TIOError io) {
-          result.io = io;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_deleteColumnFamily();
       }
     }
 
-    public static class getRegionLocation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getRegionLocation_args> {
-      public getRegionLocation() {
-        super("getRegionLocation");
+    public void modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      modifyColumnFamily_call method_call = new modifyColumnFamily_call(tableName, column, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class modifyColumnFamily_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableName tableName;
+      private TColumnFamilyDescriptor column;
+      public modifyColumnFamily_call(TTableName tableName, TColumnFamilyDescriptor column, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tableName = tableName;
+        this.column = column;
       }
 
-      public getRegionLocation_args getEmptyArgsInstance() {
-        return new getRegionLocation_args();
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("modifyColumnFamily", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        modifyColumnFamily_args args = new modifyColumnFamily_args();
+        args.setTableName(tableName);
+        args.setColumn(column);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_modifyColumnFamily();
       }
+    }
 
-      public getRegionLocation_result getResult(I iface, getRegionLocation_args args) throws org.apache.thrift.TException {
-        getRegionLocation_result result = new getRegionLocation_result();
-        try {
-          result.success = iface.getRegionLocation(args.table, args.row, args.reload);
-        } catch (TIOError io) {
-          result.io = io;
+    public void modifyTable(TTableDescriptor desc, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      modifyTable_call method_call = new modifyTable_call(desc, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class modifyTable_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TTableDescriptor desc;
+      public modifyTable_call(TTableDescriptor desc, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.desc = desc;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("modifyTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        modifyTable_args args = new modifyTable_args();
+        args.setDesc(desc);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_modifyTable();
       }
     }
 
-    public static class getAllRegionLocations<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getAllRegionLocations_args> {
-      public getAllRegionLocations() {
-        super("getAllRegionLocations");
+    public void createNamespace(TNamespaceDescriptor namespaceDesc, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      createNamespace_call method_call = new createNamespace_call(namespaceDesc, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class createNamespace_call extends org.apache.thrift.async.TAsyncMethodCall {
+  

<TRUNCATED>

[16/47] hbase git commit: HBASE-21514 Refactor CacheConfig

Posted by zh...@apache.org.
HBASE-21514 Refactor CacheConfig


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

Branch: refs/heads/HBASE-21512
Commit: 1971d02e725341fdee79b7ee2308a9870debe2f6
Parents: 68b5df0
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Nov 29 10:30:45 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Tue Dec 18 13:43:30 2018 +0800

----------------------------------------------------------------------
 .../tmpl/regionserver/BlockCacheTmpl.jamon      |  36 +-
 .../tmpl/regionserver/BlockCacheViewTmpl.jamon  |   3 +-
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon  |   5 +-
 .../hbase/io/hfile/BlockCacheFactory.java       | 226 +++++++++
 .../hadoop/hbase/io/hfile/CacheConfig.java      | 499 ++++---------------
 .../hbase/io/hfile/CombinedBlockCache.java      |  12 +
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |  17 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  | 176 +++----
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |   9 +-
 .../hbase/io/hfile/bucket/BucketAllocator.java  |   4 +-
 .../assignment/MergeTableRegionsProcedure.java  |  15 +-
 .../assignment/SplitTableRegionProcedure.java   |  15 +-
 .../apache/hadoop/hbase/mob/MobCacheConfig.java |  64 ---
 .../apache/hadoop/hbase/mob/MobFileCache.java   |   5 +-
 .../hadoop/hbase/regionserver/HMobStore.java    |  28 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  40 +-
 .../hbase/regionserver/HRegionServer.java       |  55 +-
 .../hadoop/hbase/regionserver/HStore.java       |   2 +-
 .../hbase/regionserver/HeapMemoryManager.java   |  30 +-
 .../MetricsRegionServerWrapperImpl.java         | 267 +++-------
 .../hbase/regionserver/RSRpcServices.java       |   3 +-
 .../regionserver/RegionServerServices.java      |  15 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |  48 +-
 .../hadoop/hbase/MockRegionServerServices.java  |  13 +
 ...estAvoidCellReferencesIntoShippedBlocks.java |   4 +-
 .../client/TestBlockEvictionFromClient.java     |  20 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   3 +-
 .../hbase/io/encoding/TestEncodedSeekers.java   |  26 +-
 .../hbase/io/hfile/TestBlockCacheReporting.java |  47 +-
 .../hadoop/hbase/io/hfile/TestCacheConfig.java  |  53 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  34 +-
 .../io/hfile/TestForceCacheImportantBlocks.java |  22 +-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |   9 +-
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |   3 +-
 .../hbase/io/hfile/TestHFileBlockIndex.java     |  10 +-
 .../hfile/TestLazyDataBlockDecompression.java   |  20 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |  22 +-
 .../io/hfile/TestScannerFromBucketCache.java    |  58 +--
 .../TestScannerSelectionUsingKeyRange.java      |   5 +-
 .../io/hfile/TestScannerSelectionUsingTTL.java  |  31 +-
 .../hadoop/hbase/master/MockRegionServer.java   |  13 +
 .../hbase/master/TestMasterNotCarryTable.java   |   8 +-
 .../hadoop/hbase/mob/TestMobFileCache.java      |  22 +-
 .../regionserver/DataBlockEncodingTool.java     |   7 +-
 .../EncodedSeekPerformanceTest.java             |   2 +-
 .../hbase/regionserver/TestAtomicOperation.java |  10 +-
 .../hbase/regionserver/TestBlocksRead.java      |  66 +--
 .../hbase/regionserver/TestBlocksScanned.java   |  38 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |   6 +-
 .../regionserver/TestClearRegionBlockCache.java |  46 +-
 .../regionserver/TestCompoundBloomFilter.java   |  16 +-
 .../hbase/regionserver/TestHMobStore.java       |  84 ++--
 .../hbase/regionserver/TestHStoreFile.java      |  14 +-
 .../regionserver/TestMobStoreCompaction.java    |  20 +-
 .../regionserver/TestMultiColumnScanner.java    |  15 +-
 .../hbase/regionserver/TestRSStatusServlet.java |   9 +-
 .../hbase/regionserver/TestRecoveredEdits.java  |  38 +-
 .../regionserver/TestRowPrefixBloomFilter.java  |   2 +-
 .../regionserver/TestSecureBulkLoadManager.java |   2 +-
 59 files changed, 1096 insertions(+), 1276 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
index 5ea5bcc..a18e6d4 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
@@ -20,17 +20,12 @@ Template for rendering Block Cache tabs in RegionServer Status page.
 <%args>
 CacheConfig cacheConfig;
 Configuration config;
+BlockCache bc;
 </%args>
 <%java>
-  BlockCache bc = cacheConfig == null? null: cacheConfig.getBlockCache();
-  String bcUrl = null;
-  String bcName = null;
-  if (bc != null) {
-    bcUrl = "http://hbase.apache.org/devapidocs/" + bc.getClass().getName().replaceAll("\\.", "/") + ".html";
-    bcName = bc.getClass().getSimpleName();
-  }
-  BlockCache [] bcs = cacheConfig == null? null: cacheConfig.getBlockCache() == null? null: cacheConfig.getBlockCache().getBlockCaches();
-  // If more than one bc, show evictions in each bc listing so can compare
+  String bcUrl = bc == null ? null : "http://hbase.apache.org/devapidocs/" + bc.getClass().getName().replaceAll("\\.", "/") + ".html";
+  String bcName = bc == null ? null : bc.getClass().getSimpleName();
+  BlockCache [] bcs = bc == null ? null : bc.getBlockCaches();
   boolean evictions = bcs != null && bcs.length > 1;
 </%java>
 <%import>
@@ -57,13 +52,13 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
     </ul>
     <div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
         <div class="tab-pane active" id="tab_bc_baseInfo">
-            <& bc_baseInfo; cacheConfig = cacheConfig; bcUrl = bcUrl; bcName = bcName;  &>
+            <& bc_baseInfo; bc = bc; bcUrl = bcUrl; bcName = bcName;  &>
         </div>
         <div class="tab-pane" id="tab_bc_config">
             <& bc_config; cacheConfig = cacheConfig &>
         </div>
         <div class="tab-pane" id="tab_bc_stats">
-            <& bc_stats; cacheConfig = cacheConfig &>
+            <& bc_stats; bc = bc &>
         </div>
         <div class="tab-pane" id="tab_bc_l1">
             <& bc_l; bc = bcs == null? bc: bcs[0]; name = "L1"; evictions = evictions; &>
@@ -76,12 +71,11 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
 <%def bc_baseInfo>
 <%args>
-    CacheConfig cacheConfig;
+    BlockCache bc;
     String bcUrl;
     String bcName;
 </%args>
 <%java>
-  BlockCache bc = cacheConfig == null? null: cacheConfig.getBlockCache();
   BlockCache [] bcs = bc == null? null: bc.getBlockCaches();
   String bcl1Url = null;
   String bcl1Name = null;
@@ -229,10 +223,10 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
 <%def bc_stats>
 <%args>
-    CacheConfig cacheConfig;
+    BlockCache bc;
 </%args>
-<%if cacheConfig == null || cacheConfig.getBlockCache() == null %>
-<p>CacheConfig is null</p>
+<%if bc == null %>
+<p>BlockCache is null</p>
 <%else>
 <table class="table table-striped">
     <tr>
@@ -242,23 +236,23 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
     </tr>
     <tr>
         <td>Size</td>
-        <td><% TraditionalBinaryPrefix.long2String(cacheConfig.getBlockCache().getCurrentSize(),
+        <td><% TraditionalBinaryPrefix.long2String(bc.getCurrentSize(),
             "B", 1) %></td>
         <td>Current size of block cache in use</td>
     </tr>
     <tr>
         <td>Free</td>
-        <td><% TraditionalBinaryPrefix.long2String(cacheConfig.getBlockCache().getFreeSize(),
+        <td><% TraditionalBinaryPrefix.long2String(bc.getFreeSize(),
             "B", 1) %></td>
         <td>The total free memory currently available to store more cache entries</td>
     </tr>
     <tr>
         <td>Count</td>
-        <td><% String.format("%,d", cacheConfig.getBlockCache().getBlockCount()) %></td>
+        <td><% String.format("%,d", bc.getBlockCount()) %></td>
         <td>Number of blocks in block cache</td>
     </tr>
-    <& evictions_tmpl; bc = cacheConfig.getBlockCache(); &>
-    <& hits_tmpl; bc = cacheConfig.getBlockCache(); &>
+    <& evictions_tmpl; bc = bc; &>
+    <& hits_tmpl; bc = bc; &>
 </table>
 <p>If block cache is made up of more than one cache -- i.e. a L1 and a L2 -- then the above
 are combined counts. Request count is sum of hits and misses.</p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
index c6d7a61..d5a9aa5 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
@@ -23,6 +23,7 @@ CacheConfig cacheConfig;
 Configuration conf;
 String bcn;
 String bcv;
+BlockCache blockCache;
 </%args>
 <%import>
 java.util.*;
@@ -39,7 +40,7 @@ org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket;
 org.apache.hadoop.util.StringUtils;
 </%import>
 <%java>
-  BlockCache bc = cacheConfig == null ? null : cacheConfig.getBlockCache();
+  BlockCache bc = blockCache;
   BlockCache [] bcs = bc == null ? null : bc.getBlockCaches();
   if (bcn.equals("L1")) {
     bc = bcs == null || bcs.length == 0? bc: bcs[0];

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
index 646d835..e3c38b9 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
@@ -29,13 +29,14 @@ org.apache.hadoop.hbase.regionserver.HRegionServer;
 org.apache.hadoop.hbase.client.RegionInfo;
 org.apache.hadoop.hbase.ServerName;
 org.apache.hadoop.hbase.HBaseConfiguration;
+org.apache.hadoop.hbase.io.hfile.CacheConfig;
 org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
 org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 </%import>
 <%doc>If json AND bcn is NOT an empty string presume it a block cache view request.</%doc>
 <%if format.equals("json") && bcn != null && bcn.length() > 0  %>
-  <& BlockCacheViewTmpl; conf = regionServer.getConfiguration(); cacheConfig = regionServer.getCacheConfig(); bcn = bcn; bcv = bcv;  &>
+  <& BlockCacheViewTmpl; conf = regionServer.getConfiguration(); cacheConfig = new CacheConfig(regionServer.getConfiguration()); bcn = bcn; bcv = bcv; blockCache = regionServer.getBlockCache().orElse(null)  &>
   <%java return; %>
 <%elseif format.equals("json") %>
   <& ../common/TaskMonitorTmpl; filter = filter; format = "json" &>
@@ -109,7 +110,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 
     <section>
     <h2>Block Cache</h2>
-    <& BlockCacheTmpl; cacheConfig = regionServer.getCacheConfig(); config = regionServer.getConfiguration() &>
+    <& BlockCacheTmpl; cacheConfig = new CacheConfig(regionServer.getConfiguration()); config = regionServer.getConfiguration(); bc = regionServer.getBlockCache().orElse(null) &>
     </section>
 
     <section>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java
new file mode 100644
index 0000000..98b3c4f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java
@@ -0,0 +1,226 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.hfile;
+
+import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY;
+import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_SIZE_KEY;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
+import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public final class BlockCacheFactory {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BlockCacheFactory.class.getName());
+
+  /**
+   * Configuration keys for Bucket cache
+   */
+
+  /**
+   * If the chosen ioengine can persist its state across restarts, the path to the file to persist
+   * to. This file is NOT the data file. It is a file into which we will serialize the map of
+   * what is in the data file. For example, if you pass the following argument as
+   * BUCKET_CACHE_IOENGINE_KEY ("hbase.bucketcache.ioengine"),
+   * <code>file:/tmp/bucketcache.data </code>, then we will write the bucketcache data to the file
+   * <code>/tmp/bucketcache.data</code> but the metadata on where the data is in the supplied file
+   * is an in-memory map that needs to be persisted across restarts. Where to store this
+   * in-memory state is what you supply here: e.g. <code>/tmp/bucketcache.map</code>.
+   */
+  public static final String BUCKET_CACHE_PERSISTENT_PATH_KEY = "hbase.bucketcache.persistent.path";
+
+  public static final String BUCKET_CACHE_WRITER_THREADS_KEY = "hbase.bucketcache.writer.threads";
+
+  public static final String BUCKET_CACHE_WRITER_QUEUE_KEY = "hbase.bucketcache.writer.queuelength";
+
+  /**
+   * A comma-delimited array of values for use as bucket sizes.
+   */
+  public static final String BUCKET_CACHE_BUCKETS_KEY = "hbase.bucketcache.bucket.sizes";
+
+  /**
+   * Defaults for Bucket cache
+   */
+  public static final int DEFAULT_BUCKET_CACHE_WRITER_THREADS = 3;
+  public static final int DEFAULT_BUCKET_CACHE_WRITER_QUEUE = 64;
+
+  /**
+   * The target block size used by blockcache instances. Defaults to
+   * {@link HConstants#DEFAULT_BLOCKSIZE}.
+   * TODO: this config point is completely wrong, as it's used to determine the
+   * target block size of BlockCache instances. Rename.
+   */
+  public static final String BLOCKCACHE_BLOCKSIZE_KEY = "hbase.offheapcache.minblocksize";
+
+  private static final String EXTERNAL_BLOCKCACHE_KEY = "hbase.blockcache.use.external";
+  private static final boolean EXTERNAL_BLOCKCACHE_DEFAULT = false;
+
+  private static final String EXTERNAL_BLOCKCACHE_CLASS_KEY = "hbase.blockcache.external.class";
+
+  private BlockCacheFactory() {
+  }
+
+  public static BlockCache createBlockCache(Configuration conf) {
+    LruBlockCache onHeapCache = createOnHeapCache(conf);
+    if (onHeapCache == null) {
+      return null;
+    }
+    boolean useExternal = conf.getBoolean(EXTERNAL_BLOCKCACHE_KEY, EXTERNAL_BLOCKCACHE_DEFAULT);
+    if (useExternal) {
+      BlockCache l2CacheInstance = createExternalBlockcache(conf);
+      return l2CacheInstance == null ?
+          onHeapCache :
+          new InclusiveCombinedBlockCache(onHeapCache, l2CacheInstance);
+    } else {
+      // otherwise use the bucket cache.
+      BucketCache bucketCache = createBucketCache(conf);
+      if (!conf.getBoolean("hbase.bucketcache.combinedcache.enabled", true)) {
+        // Non combined mode is off from 2.0
+        LOG.warn(
+            "From HBase 2.0 onwards only combined mode of LRU cache and bucket cache is available");
+      }
+      return bucketCache == null ? onHeapCache : new CombinedBlockCache(onHeapCache, bucketCache);
+    }
+  }
+
+  private static LruBlockCache createOnHeapCache(final Configuration c) {
+    final long cacheSize = MemorySizeUtil.getOnHeapCacheSize(c);
+    if (cacheSize < 0) {
+      return null;
+    }
+    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
+    LOG.info(
+        "Allocating onheap LruBlockCache size=" + StringUtils.byteDesc(cacheSize) + ", blockSize="
+            + StringUtils.byteDesc(blockSize));
+    return new LruBlockCache(cacheSize, blockSize, true, c);
+  }
+
+  /**
+   * Enum of all built in external block caches.
+   * This is used for config.
+   */
+  private static enum ExternalBlockCaches {
+    memcached("org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache");
+    // TODO(eclark): Consider more. Redis, etc.
+    Class<? extends BlockCache> clazz;
+    ExternalBlockCaches(String clazzName) {
+      try {
+        clazz = (Class<? extends BlockCache>) Class.forName(clazzName);
+      } catch (ClassNotFoundException cnef) {
+        clazz = null;
+      }
+    }
+    ExternalBlockCaches(Class<? extends BlockCache> clazz) {
+      this.clazz = clazz;
+    }
+  }
+
+  private static BlockCache createExternalBlockcache(Configuration c) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Trying to use External l2 cache");
+    }
+    Class klass = null;
+
+    // Get the class, from the config. s
+    try {
+      klass = ExternalBlockCaches
+          .valueOf(c.get(EXTERNAL_BLOCKCACHE_CLASS_KEY, "memcache")).clazz;
+    } catch (IllegalArgumentException exception) {
+      try {
+        klass = c.getClass(EXTERNAL_BLOCKCACHE_CLASS_KEY, Class.forName(
+            "org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache"));
+      } catch (ClassNotFoundException e) {
+        return null;
+      }
+    }
+
+    // Now try and create an instance of the block cache.
+    try {
+      LOG.info("Creating external block cache of type: " + klass);
+      return (BlockCache) ReflectionUtils.newInstance(klass, c);
+    } catch (Exception e) {
+      LOG.warn("Error creating external block cache", e);
+    }
+    return null;
+
+  }
+
+  private static BucketCache createBucketCache(Configuration c) {
+    // Check for L2.  ioengine name must be non-null.
+    String bucketCacheIOEngineName = c.get(BUCKET_CACHE_IOENGINE_KEY, null);
+    if (bucketCacheIOEngineName == null || bucketCacheIOEngineName.length() <= 0) {
+      return null;
+    }
+
+    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
+    final long bucketCacheSize = MemorySizeUtil.getBucketCacheSize(c);
+    if (bucketCacheSize <= 0) {
+      throw new IllegalStateException("bucketCacheSize <= 0; Check " +
+          BUCKET_CACHE_SIZE_KEY + " setting and/or server java heap size");
+    }
+    if (c.get("hbase.bucketcache.percentage.in.combinedcache") != null) {
+      LOG.warn("Configuration 'hbase.bucketcache.percentage.in.combinedcache' is no longer "
+          + "respected. See comments in http://hbase.apache.org/book.html#_changes_of_note");
+    }
+    int writerThreads = c.getInt(BUCKET_CACHE_WRITER_THREADS_KEY,
+        DEFAULT_BUCKET_CACHE_WRITER_THREADS);
+    int writerQueueLen = c.getInt(BUCKET_CACHE_WRITER_QUEUE_KEY,
+        DEFAULT_BUCKET_CACHE_WRITER_QUEUE);
+    String persistentPath = c.get(BUCKET_CACHE_PERSISTENT_PATH_KEY);
+    String[] configuredBucketSizes = c.getStrings(BUCKET_CACHE_BUCKETS_KEY);
+    int [] bucketSizes = null;
+    if (configuredBucketSizes != null) {
+      bucketSizes = new int[configuredBucketSizes.length];
+      for (int i = 0; i < configuredBucketSizes.length; i++) {
+        int bucketSize = Integer.parseInt(configuredBucketSizes[i].trim());
+        if (bucketSize % 256 != 0) {
+          // We need all the bucket sizes to be multiples of 256. Having all the configured bucket
+          // sizes to be multiples of 256 will ensure that the block offsets within buckets,
+          // that are calculated, will also be multiples of 256.
+          // See BucketEntry where offset to each block is represented using 5 bytes (instead of 8
+          // bytes long). We would like to save heap overhead as less as possible.
+          throw new IllegalArgumentException("Illegal value: " + bucketSize + " configured for '"
+              + BUCKET_CACHE_BUCKETS_KEY + "'. All bucket sizes to be multiples of 256");
+        }
+        bucketSizes[i] = bucketSize;
+      }
+    }
+    BucketCache bucketCache = null;
+    try {
+      int ioErrorsTolerationDuration = c.getInt(
+          "hbase.bucketcache.ioengine.errors.tolerated.duration",
+          BucketCache.DEFAULT_ERROR_TOLERATION_DURATION);
+      // Bucket cache logs its stats on creation internal to the constructor.
+      bucketCache = new BucketCache(bucketCacheIOEngineName,
+          bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath,
+          ioErrorsTolerationDuration, c);
+    } catch (IOException ioex) {
+      LOG.error("Can't instantiate bucket cache", ioex); throw new RuntimeException(ioex);
+    }
+    return bucketCache;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 a022552..cd9303d 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
@@ -17,26 +17,17 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY;
-import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_SIZE_KEY;
-
-import java.io.IOException;
+import java.util.Optional;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
-import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
-import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
-
 /**
  * Stores all of the cache objects and configuration for a single HFile.
  */
@@ -44,7 +35,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 public class CacheConfig {
   private static final Logger LOG = LoggerFactory.getLogger(CacheConfig.class.getName());
 
-
   /**
    * Disabled cache configuration
    */
@@ -60,109 +50,38 @@ public class CacheConfig {
    * Configuration key to cache data blocks on write. There are separate
    * switches for bloom blocks and non-root index blocks.
    */
-  public static final String CACHE_BLOCKS_ON_WRITE_KEY =
-      "hbase.rs.cacheblocksonwrite";
+  public static final String CACHE_BLOCKS_ON_WRITE_KEY = "hbase.rs.cacheblocksonwrite";
 
   /**
    * Configuration key to cache leaf and intermediate-level index blocks on
    * write.
    */
-  public static final String CACHE_INDEX_BLOCKS_ON_WRITE_KEY =
-      "hfile.block.index.cacheonwrite";
+  public static final String CACHE_INDEX_BLOCKS_ON_WRITE_KEY = "hfile.block.index.cacheonwrite";
 
   /**
    * Configuration key to cache compound bloom filter blocks on write.
    */
-  public static final String CACHE_BLOOM_BLOCKS_ON_WRITE_KEY =
-      "hfile.block.bloom.cacheonwrite";
+  public static final String CACHE_BLOOM_BLOCKS_ON_WRITE_KEY = "hfile.block.bloom.cacheonwrite";
 
   /**
    * Configuration key to cache data blocks in compressed and/or encrypted format.
    */
-  public static final String CACHE_DATA_BLOCKS_COMPRESSED_KEY =
-      "hbase.block.data.cachecompressed";
+  public static final String CACHE_DATA_BLOCKS_COMPRESSED_KEY = "hbase.block.data.cachecompressed";
 
   /**
    * Configuration key to evict all blocks of a given file from the block cache
    * when the file is closed.
    */
-  public static final String EVICT_BLOCKS_ON_CLOSE_KEY =
-      "hbase.rs.evictblocksonclose";
-
-  /**
-   * Configuration keys for Bucket cache
-   */
-
-  /**
-   * If the chosen ioengine can persist its state across restarts, the path to the file to persist
-   * to. This file is NOT the data file. It is a file into which we will serialize the map of
-   * what is in the data file. For example, if you pass the following argument as
-   * BUCKET_CACHE_IOENGINE_KEY ("hbase.bucketcache.ioengine"),
-   * <code>file:/tmp/bucketcache.data </code>, then we will write the bucketcache data to the file
-   * <code>/tmp/bucketcache.data</code> but the metadata on where the data is in the supplied file
-   * is an in-memory map that needs to be persisted across restarts. Where to store this
-   * in-memory state is what you supply here: e.g. <code>/tmp/bucketcache.map</code>.
-   */
-  public static final String BUCKET_CACHE_PERSISTENT_PATH_KEY =
-      "hbase.bucketcache.persistent.path";
-
-  public static final String BUCKET_CACHE_WRITER_THREADS_KEY = "hbase.bucketcache.writer.threads";
-  public static final String BUCKET_CACHE_WRITER_QUEUE_KEY =
-      "hbase.bucketcache.writer.queuelength";
-
-  /**
-   * A comma-delimited array of values for use as bucket sizes.
-   */
-  public static final String BUCKET_CACHE_BUCKETS_KEY = "hbase.bucketcache.bucket.sizes";
+  public static final String EVICT_BLOCKS_ON_CLOSE_KEY = "hbase.rs.evictblocksonclose";
 
   /**
-   * Defaults for Bucket cache
-   */
-  public static final int DEFAULT_BUCKET_CACHE_WRITER_THREADS = 3;
-  public static final int DEFAULT_BUCKET_CACHE_WRITER_QUEUE = 64;
-
- /**
    * Configuration key to prefetch all blocks of a given file into the block cache
    * when the file is opened.
    */
-  public static final String PREFETCH_BLOCKS_ON_OPEN_KEY =
-      "hbase.rs.prefetchblocksonopen";
-
-  /**
-   * The target block size used by blockcache instances. Defaults to
-   * {@link HConstants#DEFAULT_BLOCKSIZE}.
-   * TODO: this config point is completely wrong, as it's used to determine the
-   * target block size of BlockCache instances. Rename.
-   */
-  public static final String BLOCKCACHE_BLOCKSIZE_KEY = "hbase.offheapcache.minblocksize";
-
-  private static final String EXTERNAL_BLOCKCACHE_KEY = "hbase.blockcache.use.external";
-  private static final boolean EXTERNAL_BLOCKCACHE_DEFAULT = false;
+  public static final String PREFETCH_BLOCKS_ON_OPEN_KEY = "hbase.rs.prefetchblocksonopen";
 
-  private static final String EXTERNAL_BLOCKCACHE_CLASS_KEY = "hbase.blockcache.external.class";
-  private static final String DROP_BEHIND_CACHE_COMPACTION_KEY =
+  public static final String DROP_BEHIND_CACHE_COMPACTION_KEY =
       "hbase.hfile.drop.behind.compaction";
-  private static final boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true;
-
-  /**
-   * Enum of all built in external block caches.
-   * This is used for config.
-   */
-  private static enum ExternalBlockCaches {
-    memcached("org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache");
-    // TODO(eclark): Consider more. Redis, etc.
-    Class<? extends BlockCache> clazz;
-    ExternalBlockCaches(String clazzName) {
-      try {
-        clazz = (Class<? extends BlockCache>) Class.forName(clazzName);
-      } catch (ClassNotFoundException cnef) {
-        clazz = null;
-      }
-    }
-    ExternalBlockCaches(Class<? extends BlockCache> clazz) {
-      this.clazz = clazz;
-    }
-  }
 
   // Defaults
   public static final boolean DEFAULT_CACHE_DATA_ON_READ = true;
@@ -173,9 +92,7 @@ public class CacheConfig {
   public static final boolean DEFAULT_EVICT_ON_CLOSE = false;
   public static final boolean DEFAULT_CACHE_DATA_COMPRESSED = false;
   public static final boolean DEFAULT_PREFETCH_ON_OPEN = false;
-
-  /** Local reference to the block cache, null if completely disabled */
-  private final BlockCache blockCache;
+  public static final boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true;
 
   /**
    * Whether blocks should be cached on read (default is on if there is a
@@ -183,7 +100,7 @@ public class CacheConfig {
    * If off we will STILL cache meta blocks; i.e. INDEX and BLOOM types.
    * This cannot be disabled.
    */
-  private boolean cacheDataOnRead;
+  private final boolean cacheDataOnRead;
 
   /** Whether blocks should be flagged as in-memory when being cached */
   private final boolean inMemory;
@@ -208,89 +125,54 @@ public class CacheConfig {
 
   private final boolean dropBehindCompaction;
 
-  /**
-   * Create a cache configuration using the specified configuration object and
-   * family descriptor.
-   * @param conf hbase configuration
-   * @param family column family configuration
-   */
-  public CacheConfig(Configuration conf, ColumnFamilyDescriptor family) {
-    this(GLOBAL_BLOCK_CACHE_INSTANCE,
-        conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ)
-           && family.isBlockCacheEnabled(),
-        family.isInMemory(),
-        // For the following flags we enable them regardless of per-schema settings
-        // if they are enabled in the global configuration.
-        conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY,
-            DEFAULT_CACHE_DATA_ON_WRITE) || family.isCacheDataOnWrite(),
-        conf.getBoolean(CACHE_INDEX_BLOCKS_ON_WRITE_KEY,
-            DEFAULT_CACHE_INDEXES_ON_WRITE) || family.isCacheIndexesOnWrite(),
-        conf.getBoolean(CACHE_BLOOM_BLOCKS_ON_WRITE_KEY,
-            DEFAULT_CACHE_BLOOMS_ON_WRITE) || family.isCacheBloomsOnWrite(),
-        conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY,
-            DEFAULT_EVICT_ON_CLOSE) || family.isEvictBlocksOnClose(),
-        conf.getBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, DEFAULT_CACHE_DATA_COMPRESSED),
-        conf.getBoolean(PREFETCH_BLOCKS_ON_OPEN_KEY,
-            DEFAULT_PREFETCH_ON_OPEN) || family.isPrefetchBlocksOnOpen(),
-        conf.getBoolean(DROP_BEHIND_CACHE_COMPACTION_KEY, DROP_BEHIND_CACHE_COMPACTION_DEFAULT)
-     );
-    LOG.info("Created cacheConfig for " + family.getNameAsString() + ": " + this);
-  }
+  // Local reference to the block cache
+  private final BlockCache blockCache;
 
   /**
    * Create a cache configuration using the specified configuration object and
-   * defaults for family level settings. Only use if no column family context. Prefer
-   * {@link CacheConfig#CacheConfig(Configuration, ColumnFamilyDescriptor)}
-   * @see #CacheConfig(Configuration, ColumnFamilyDescriptor)
+   * defaults for family level settings. Only use if no column family context.
    * @param conf hbase configuration
    */
   public CacheConfig(Configuration conf) {
-    this(GLOBAL_BLOCK_CACHE_INSTANCE,
-        conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ),
-        DEFAULT_IN_MEMORY, // This is a family-level setting so can't be set
-        // strictly from conf
-        conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_DATA_ON_WRITE),
-        conf.getBoolean(CACHE_INDEX_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_INDEXES_ON_WRITE),
-        conf.getBoolean(CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_BLOOMS_ON_WRITE),
-        conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY, DEFAULT_EVICT_ON_CLOSE),
-        conf.getBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, DEFAULT_CACHE_DATA_COMPRESSED),
-        conf.getBoolean(PREFETCH_BLOCKS_ON_OPEN_KEY, DEFAULT_PREFETCH_ON_OPEN),
-        conf.getBoolean(DROP_BEHIND_CACHE_COMPACTION_KEY, DROP_BEHIND_CACHE_COMPACTION_DEFAULT));
-    LOG.info("Created cacheConfig: " + this);
+    this(conf, null);
+  }
+
+  public CacheConfig(Configuration conf, BlockCache blockCache) {
+    this(conf, null, blockCache);
   }
 
   /**
-   * 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).
-   * @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
-   * @param cacheBloomsOnWrite whether blooms should be cached on write
-   * @param evictOnClose whether blocks should be evicted when HFile is closed
-   * @param cacheDataCompressed whether to store blocks as compressed in the cache
-   * @param prefetchOnOpen whether to prefetch blocks upon open
-   * @param dropBehindCompaction indicate that we should set drop behind to true when open a store
-   *          file reader for compaction
+   * Create a cache configuration using the specified configuration object and
+   * family descriptor.
+   * @param conf hbase configuration
+   * @param family column family configuration
    */
-  @VisibleForTesting
-  CacheConfig(final BlockCache blockCache,
-      final boolean cacheDataOnRead, final boolean inMemory,
-      final boolean cacheDataOnWrite, final boolean cacheIndexesOnWrite,
-      final boolean cacheBloomsOnWrite, final boolean evictOnClose,
-      final boolean cacheDataCompressed, final boolean prefetchOnOpen,
-      final boolean dropBehindCompaction) {
+  public CacheConfig(Configuration conf, ColumnFamilyDescriptor family, BlockCache blockCache) {
+    this.cacheDataOnRead = conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ) &&
+        (family == null ? true : family.isBlockCacheEnabled());
+    this.inMemory = family == null ? DEFAULT_IN_MEMORY : family.isInMemory();
+    this.cacheDataCompressed =
+        conf.getBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, DEFAULT_CACHE_DATA_COMPRESSED);
+    this.dropBehindCompaction =
+        conf.getBoolean(DROP_BEHIND_CACHE_COMPACTION_KEY, DROP_BEHIND_CACHE_COMPACTION_DEFAULT);
+    // For the following flags we enable them regardless of per-schema settings
+    // if they are enabled in the global configuration.
+    this.cacheDataOnWrite =
+        conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_DATA_ON_WRITE) ||
+            (family == null ? false : family.isCacheDataOnWrite());
+    this.cacheIndexesOnWrite =
+        conf.getBoolean(CACHE_INDEX_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_INDEXES_ON_WRITE) ||
+            (family == null ? false : family.isCacheIndexesOnWrite());
+    this.cacheBloomsOnWrite =
+        conf.getBoolean(CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_BLOOMS_ON_WRITE) ||
+            (family == null ? false : family.isCacheBloomsOnWrite());
+    this.evictOnClose = conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY, DEFAULT_EVICT_ON_CLOSE) ||
+        (family == null ? false : family.isEvictBlocksOnClose());
+    this.prefetchOnOpen = conf.getBoolean(PREFETCH_BLOCKS_ON_OPEN_KEY, DEFAULT_PREFETCH_ON_OPEN) ||
+        (family == null ? false : family.isPrefetchBlocksOnOpen());
     this.blockCache = blockCache;
-    this.cacheDataOnRead = cacheDataOnRead;
-    this.inMemory = inMemory;
-    this.cacheDataOnWrite = cacheDataOnWrite;
-    this.cacheIndexesOnWrite = cacheIndexesOnWrite;
-    this.cacheBloomsOnWrite = cacheBloomsOnWrite;
-    this.evictOnClose = evictOnClose;
-    this.cacheDataCompressed = cacheDataCompressed;
-    this.prefetchOnOpen = prefetchOnOpen;
-    this.dropBehindCompaction = dropBehindCompaction;
+    LOG.info("Created cacheConfig: " + this + (family == null ? "" : " for family " + family) +
+        " with blockCache=" + blockCache);
   }
 
   /**
@@ -298,30 +180,29 @@ public class CacheConfig {
    * @param cacheConf
    */
   public CacheConfig(CacheConfig cacheConf) {
-    this(cacheConf.blockCache, cacheConf.cacheDataOnRead, cacheConf.inMemory,
-        cacheConf.cacheDataOnWrite, cacheConf.cacheIndexesOnWrite,
-        cacheConf.cacheBloomsOnWrite, cacheConf.evictOnClose,
-        cacheConf.cacheDataCompressed, cacheConf.prefetchOnOpen,
-        cacheConf.dropBehindCompaction);
+    this.cacheDataOnRead = cacheConf.cacheDataOnRead;
+    this.inMemory = cacheConf.inMemory;
+    this.cacheDataOnWrite = cacheConf.cacheDataOnWrite;
+    this.cacheIndexesOnWrite = cacheConf.cacheIndexesOnWrite;
+    this.cacheBloomsOnWrite = cacheConf.cacheBloomsOnWrite;
+    this.evictOnClose = cacheConf.evictOnClose;
+    this.cacheDataCompressed = cacheConf.cacheDataCompressed;
+    this.prefetchOnOpen = cacheConf.prefetchOnOpen;
+    this.dropBehindCompaction = cacheConf.dropBehindCompaction;
+    this.blockCache = cacheConf.blockCache;
   }
 
   private CacheConfig() {
-    this(null, false, false, false, false, false, false, false, false, false);
-  }
-
-  /**
-   * Checks whether the block cache is enabled.
-   */
-  public boolean isBlockCacheEnabled() {
-    return this.blockCache != null;
-  }
-
-  /**
-   * Returns the block cache.
-   * @return the block cache, or null if caching is completely disabled
-   */
-  public BlockCache getBlockCache() {
-    return this.blockCache;
+    this.cacheDataOnRead = false;
+    this.inMemory = false;
+    this.cacheDataOnWrite = false;
+    this.cacheIndexesOnWrite = false;
+    this.cacheBloomsOnWrite = false;
+    this.evictOnClose = false;
+    this.cacheDataCompressed = false;
+    this.prefetchOnOpen = false;
+    this.dropBehindCompaction = false;
+    this.blockCache = null;
   }
 
   /**
@@ -330,7 +211,7 @@ public class CacheConfig {
    * @return true if blocks should be cached on read, false if not
    */
   public boolean shouldCacheDataOnRead() {
-    return isBlockCacheEnabled() && cacheDataOnRead;
+    return cacheDataOnRead;
   }
 
   public boolean shouldDropBehindCompaction() {
@@ -343,20 +224,15 @@ public class CacheConfig {
    * available.
    */
   public boolean shouldCacheBlockOnRead(BlockCategory category) {
-    return isBlockCacheEnabled()
-        && (cacheDataOnRead ||
-            category == BlockCategory.INDEX ||
-            category == BlockCategory.BLOOM ||
-            (prefetchOnOpen &&
-                (category != BlockCategory.META &&
-                 category != BlockCategory.UNKNOWN)));
+    return cacheDataOnRead || category == BlockCategory.INDEX || category == BlockCategory.BLOOM ||
+        (prefetchOnOpen && (category != BlockCategory.META && category != BlockCategory.UNKNOWN));
   }
 
   /**
    * @return true if blocks in this file should be flagged as in-memory
    */
   public boolean isInMemory() {
-    return isBlockCacheEnabled() && this.inMemory;
+    return this.inMemory;
   }
 
   /**
@@ -364,11 +240,10 @@ public class CacheConfig {
    *         written, false if not
    */
   public boolean shouldCacheDataOnWrite() {
-    return isBlockCacheEnabled() && this.cacheDataOnWrite;
+    return this.cacheDataOnWrite;
   }
 
   /**
-   * Only used for testing.
    * @param cacheDataOnWrite whether data blocks should be written to the cache
    *                         when an HFile is written
    */
@@ -382,7 +257,7 @@ public class CacheConfig {
    *         is written, false if not
    */
   public boolean shouldCacheIndexesOnWrite() {
-    return isBlockCacheEnabled() && this.cacheIndexesOnWrite;
+    return this.cacheIndexesOnWrite;
   }
 
   /**
@@ -390,7 +265,7 @@ public class CacheConfig {
    *         is written, false if not
    */
   public boolean shouldCacheBloomsOnWrite() {
-    return isBlockCacheEnabled() && this.cacheBloomsOnWrite;
+    return this.cacheBloomsOnWrite;
   }
 
   /**
@@ -398,7 +273,7 @@ public class CacheConfig {
    *         reader is closed, false if not
    */
   public boolean shouldEvictOnClose() {
-    return isBlockCacheEnabled() && this.evictOnClose;
+    return this.evictOnClose;
   }
 
   /**
@@ -406,6 +281,7 @@ public class CacheConfig {
    * @param evictOnClose whether blocks should be evicted from the cache when an
    *                     HFile reader is closed
    */
+  @VisibleForTesting
   public void setEvictOnClose(boolean evictOnClose) {
     this.evictOnClose = evictOnClose;
   }
@@ -414,14 +290,13 @@ public class CacheConfig {
    * @return true if data blocks should be compressed in the cache, false if not
    */
   public boolean shouldCacheDataCompressed() {
-    return isBlockCacheEnabled() && this.cacheDataOnRead && this.cacheDataCompressed;
+    return this.cacheDataOnRead && this.cacheDataCompressed;
   }
 
   /**
    * @return true if this {@link BlockCategory} should be compressed in blockcache, false otherwise
    */
   public boolean shouldCacheCompressed(BlockCategory category) {
-    if (!isBlockCacheEnabled()) return false;
     switch (category) {
       case DATA:
         return this.cacheDataOnRead && this.cacheDataCompressed;
@@ -434,7 +309,7 @@ public class CacheConfig {
    * @return true if blocks should be prefetched into the cache on open, false if not
    */
   public boolean shouldPrefetchOnOpen() {
-    return isBlockCacheEnabled() && this.prefetchOnOpen;
+    return this.prefetchOnOpen;
   }
 
   /**
@@ -446,9 +321,6 @@ public class CacheConfig {
    * configuration.
    */
   public boolean shouldReadBlockFromCache(BlockType blockType) {
-    if (!isBlockCacheEnabled()) {
-      return false;
-    }
     if (cacheDataOnRead) {
       return true;
     }
@@ -462,7 +334,7 @@ public class CacheConfig {
       return true;
     }
     if (blockType.getCategory() == BlockCategory.BLOOM ||
-            blockType.getCategory() == BlockCategory.INDEX) {
+        blockType.getCategory() == BlockCategory.INDEX) {
       return true;
     }
     return false;
@@ -479,206 +351,21 @@ public class CacheConfig {
     return shouldCacheBlockOnRead(blockType.getCategory());
   }
 
-  @Override
-  public String toString() {
-    if (!isBlockCacheEnabled()) {
-      return "CacheConfig:disabled";
-    }
-    return "blockCache=" + getBlockCache() +
-      ", cacheDataOnRead=" + shouldCacheDataOnRead() +
-      ", cacheDataOnWrite=" + shouldCacheDataOnWrite() +
-      ", cacheIndexesOnWrite=" + shouldCacheIndexesOnWrite() +
-      ", cacheBloomsOnWrite=" + shouldCacheBloomsOnWrite() +
-      ", cacheEvictOnClose=" + shouldEvictOnClose() +
-      ", cacheDataCompressed=" + shouldCacheDataCompressed() +
-      ", prefetchOnOpen=" + shouldPrefetchOnOpen();
-  }
-
-  // Static block cache reference and methods
-
-  /**
-   * Static reference to the block cache, or null if no caching should be used
-   * at all.
-   */
-  // Clear this if in tests you'd make more than one block cache instance.
-  @VisibleForTesting
-  static BlockCache GLOBAL_BLOCK_CACHE_INSTANCE;
-  private static LruBlockCache ONHEAP_CACHE_INSTANCE = null;
-  private static BlockCache L2_CACHE_INSTANCE = null;// Can be BucketCache or External cache.
-
-  /** Boolean whether we have disabled the block cache entirely. */
-  @VisibleForTesting
-  static boolean blockCacheDisabled = false;
-
-  /**
-   * @param c Configuration to use.
-   * @return An L1 instance.  Currently an instance of LruBlockCache.
-   */
-  public static LruBlockCache getOnHeapCache(final Configuration c) {
-    return getOnHeapCacheInternal(c);
-  }
-
-  public CacheStats getOnHeapCacheStats() {
-    if (ONHEAP_CACHE_INSTANCE != null) {
-      return ONHEAP_CACHE_INSTANCE.getStats();
-    }
-    return null;
-  }
-
-  public CacheStats getL2CacheStats() {
-    if (L2_CACHE_INSTANCE != null) {
-      return L2_CACHE_INSTANCE.getStats();
-    }
-    return null;
-  }
-
-  /**
-   * @param c Configuration to use.
-   * @return An L1 instance.  Currently an instance of LruBlockCache.
-   */
-  private synchronized static LruBlockCache getOnHeapCacheInternal(final Configuration c) {
-    if (ONHEAP_CACHE_INSTANCE != null) {
-      return ONHEAP_CACHE_INSTANCE;
-    }
-    final long cacheSize = MemorySizeUtil.getOnHeapCacheSize(c);
-    if (cacheSize < 0) {
-      blockCacheDisabled = true;
-    }
-    if (blockCacheDisabled) return null;
-    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
-    LOG.info("Allocating onheap LruBlockCache size=" +
-      StringUtils.byteDesc(cacheSize) + ", blockSize=" + StringUtils.byteDesc(blockSize));
-    ONHEAP_CACHE_INSTANCE = new LruBlockCache(cacheSize, blockSize, true, c);
-    return ONHEAP_CACHE_INSTANCE;
-  }
-
-  private static BlockCache getExternalBlockcache(Configuration c) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Trying to use External l2 cache");
-    }
-    Class klass = null;
-
-    // Get the class, from the config. s
-    try {
-      klass = ExternalBlockCaches.valueOf(c.get(EXTERNAL_BLOCKCACHE_CLASS_KEY, "memcache")).clazz;
-    } catch (IllegalArgumentException exception) {
-      try {
-        klass = c.getClass(EXTERNAL_BLOCKCACHE_CLASS_KEY, Class.forName(
-            "org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache"));
-      } catch (ClassNotFoundException e) {
-        return null;
-      }
-    }
-
-    // Now try and create an instance of the block cache.
-    try {
-      LOG.info("Creating external block cache of type: " + klass);
-      return (BlockCache) ReflectionUtils.newInstance(klass, c);
-    } catch (Exception e) {
-      LOG.warn("Error creating external block cache", e);
-    }
-    return null;
-
-  }
-
-  @VisibleForTesting
-  static BucketCache getBucketCache(Configuration c) {
-    // Check for L2.  ioengine name must be non-null.
-    String bucketCacheIOEngineName = c.get(BUCKET_CACHE_IOENGINE_KEY, null);
-    if (bucketCacheIOEngineName == null || bucketCacheIOEngineName.length() <= 0) return null;
-
-    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
-    final long bucketCacheSize = MemorySizeUtil.getBucketCacheSize(c);
-    if (bucketCacheSize <= 0) {
-      throw new IllegalStateException("bucketCacheSize <= 0; Check " +
-        BUCKET_CACHE_SIZE_KEY + " setting and/or server java heap size");
-    }
-    if (c.get("hbase.bucketcache.percentage.in.combinedcache") != null) {
-      LOG.warn("Configuration 'hbase.bucketcache.percentage.in.combinedcache' is no longer "
-          + "respected. See comments in http://hbase.apache.org/book.html#_changes_of_note");
-    }
-    int writerThreads = c.getInt(BUCKET_CACHE_WRITER_THREADS_KEY,
-      DEFAULT_BUCKET_CACHE_WRITER_THREADS);
-    int writerQueueLen = c.getInt(BUCKET_CACHE_WRITER_QUEUE_KEY,
-      DEFAULT_BUCKET_CACHE_WRITER_QUEUE);
-    String persistentPath = c.get(BUCKET_CACHE_PERSISTENT_PATH_KEY);
-    String[] configuredBucketSizes = c.getStrings(BUCKET_CACHE_BUCKETS_KEY);
-    int [] bucketSizes = null;
-    if (configuredBucketSizes != null) {
-      bucketSizes = new int[configuredBucketSizes.length];
-      for (int i = 0; i < configuredBucketSizes.length; i++) {
-        int bucketSize = Integer.parseInt(configuredBucketSizes[i].trim());
-        if (bucketSize % 256 != 0) {
-          // We need all the bucket sizes to be multiples of 256. Having all the configured bucket
-          // sizes to be multiples of 256 will ensure that the block offsets within buckets,
-          // that are calculated, will also be multiples of 256.
-          // See BucketEntry where offset to each block is represented using 5 bytes (instead of 8
-          // bytes long). We would like to save heap overhead as less as possible.
-          throw new IllegalArgumentException("Illegal value: " + bucketSize + " configured for '"
-              + BUCKET_CACHE_BUCKETS_KEY + "'. All bucket sizes to be multiples of 256");
-        }
-        bucketSizes[i] = bucketSize;
-      }
-    }
-    BucketCache bucketCache = null;
-    try {
-      int ioErrorsTolerationDuration = c.getInt(
-        "hbase.bucketcache.ioengine.errors.tolerated.duration",
-        BucketCache.DEFAULT_ERROR_TOLERATION_DURATION);
-      // Bucket cache logs its stats on creation internal to the constructor.
-      bucketCache = new BucketCache(bucketCacheIOEngineName,
-        bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath,
-        ioErrorsTolerationDuration, c);
-    } catch (IOException ioex) {
-      LOG.error("Can't instantiate bucket cache", ioex); throw new RuntimeException(ioex);
-    }
-    return bucketCache;
-  }
-
   /**
-   * Returns the block cache or <code>null</code> in case none should be used.
-   * Sets GLOBAL_BLOCK_CACHE_INSTANCE
+   * Returns the block cache.
    *
-   * @param conf  The current configuration.
-   * @return The block cache or <code>null</code>.
+   * @return the block cache, or null if caching is completely disabled
    */
-  public static synchronized BlockCache instantiateBlockCache(Configuration conf) {
-    if (GLOBAL_BLOCK_CACHE_INSTANCE != null) {
-      return GLOBAL_BLOCK_CACHE_INSTANCE;
-    }
-    if (blockCacheDisabled) {
-      return null;
-    }
-    LruBlockCache onHeapCache = getOnHeapCacheInternal(conf);
-    // blockCacheDisabled is set as a side-effect of getL1Internal(), so check it again after the
-    // call.
-    if (blockCacheDisabled) {
-      return null;
-    }
-    boolean useExternal = conf.getBoolean(EXTERNAL_BLOCKCACHE_KEY, EXTERNAL_BLOCKCACHE_DEFAULT);
-    if (useExternal) {
-      L2_CACHE_INSTANCE = getExternalBlockcache(conf);
-      GLOBAL_BLOCK_CACHE_INSTANCE = L2_CACHE_INSTANCE == null ? onHeapCache
-          : new InclusiveCombinedBlockCache(onHeapCache, L2_CACHE_INSTANCE);
-    } else {
-      // otherwise use the bucket cache.
-      L2_CACHE_INSTANCE = getBucketCache(conf);
-      if (!conf.getBoolean("hbase.bucketcache.combinedcache.enabled", true)) {
-        // Non combined mode is off from 2.0
-        LOG.warn(
-            "From HBase 2.0 onwards only combined mode of LRU cache and bucket cache is available");
-      }
-      GLOBAL_BLOCK_CACHE_INSTANCE = L2_CACHE_INSTANCE == null ? onHeapCache
-          : new CombinedBlockCache(onHeapCache, L2_CACHE_INSTANCE);
-    }
-    return GLOBAL_BLOCK_CACHE_INSTANCE;
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.ofNullable(this.blockCache);
   }
 
-  // Supposed to use only from tests. Some tests want to reinit the Global block cache instance
-  @VisibleForTesting
-  static synchronized void clearGlobalInstances() {
-    ONHEAP_CACHE_INSTANCE = null;
-    L2_CACHE_INSTANCE = null;
-    GLOBAL_BLOCK_CACHE_INSTANCE = null;
+  @Override
+  public String toString() {
+    return "cacheDataOnRead=" + shouldCacheDataOnRead() + ", cacheDataOnWrite="
+        + shouldCacheDataOnWrite() + ", cacheIndexesOnWrite=" + shouldCacheIndexesOnWrite()
+        + ", cacheBloomsOnWrite=" + shouldCacheBloomsOnWrite() + ", cacheEvictOnClose="
+        + shouldEvictOnClose() + ", cacheDataCompressed=" + shouldCacheDataCompressed()
+        + ", prefetchOnOpen=" + shouldPrefetchOnOpen();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 5b17b38..b7b9c77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -152,6 +152,14 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
       this.bucketCacheStats = fcStats;
     }
 
+    public CacheStats getLruCacheStats() {
+      return this.lruCacheStats;
+    }
+
+    public CacheStats getBucketCacheStats() {
+      return this.bucketCacheStats;
+    }
+
     @Override
     public long getDataMissCount() {
       return lruCacheStats.getDataMissCount() + bucketCacheStats.getDataMissCount();
@@ -381,4 +389,8 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
     return (this.l2Cache instanceof BucketCache)
         ? ((BucketCache) this.l2Cache).getRefCount(cacheKey) : 0;
   }
+
+  public LruBlockCache getOnHeapCache() {
+    return onHeapCache;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index b91b48c..33b3f51 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -1103,9 +1103,11 @@ public class HFileBlockIndex {
           blockStream.write(midKeyMetadata);
         blockWriter.writeHeaderAndData(out);
         if (cacheConf != null) {
-          HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
-          cacheConf.getBlockCache().cacheBlock(new BlockCacheKey(nameForCaching,
-            rootLevelIndexPos, true, blockForCaching.getBlockType()), blockForCaching);
+          cacheConf.getBlockCache().ifPresent(cache -> {
+            HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
+            cache.cacheBlock(new BlockCacheKey(nameForCaching, rootLevelIndexPos, true,
+                blockForCaching.getBlockType()), blockForCaching);
+          });
         }
       }
 
@@ -1207,9 +1209,12 @@ public class HFileBlockIndex {
       blockWriter.writeHeaderAndData(out);
 
       if (getCacheOnWrite()) {
-        HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
-        cacheConf.getBlockCache().cacheBlock(new BlockCacheKey(nameForCaching,
-          beginOffset, true, blockForCaching.getBlockType()), blockForCaching);
+        cacheConf.getBlockCache().ifPresent(cache -> {
+          HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
+          cache.cacheBlock(
+              new BlockCacheKey(nameForCaching, beginOffset, true, blockForCaching.getBlockType()),
+              blockForCaching);
+        });
       }
 
       // Add intermediate index block size

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 a4a40ba..0fc9576 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
@@ -370,13 +370,16 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
   @Override
   public void returnBlock(HFileBlock block) {
-    BlockCache blockCache = this.cacheConf.getBlockCache();
-    if (blockCache != null && block != null) {
-      BlockCacheKey cacheKey = new BlockCacheKey(this.getFileContext().getHFileName(),
-          block.getOffset(), this.isPrimaryReplicaReader(), block.getBlockType());
-      blockCache.returnBlock(cacheKey, block);
+    if (block != null) {
+      this.cacheConf.getBlockCache().ifPresent(blockCache -> {
+        BlockCacheKey cacheKey =
+            new BlockCacheKey(this.getFileContext().getHFileName(), block.getOffset(),
+                this.isPrimaryReplicaReader(), block.getBlockType());
+        blockCache.returnBlock(cacheKey, block);
+      });
     }
   }
+
   /**
    * @return the first key in the file. May be null if file has no entries. Note
    *         that this is not the first row key, but rather the byte form of the
@@ -1293,69 +1296,67 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * Retrieve block from cache. Validates the retrieved block's type vs {@code expectedBlockType}
    * and its encoding vs. {@code expectedDataBlockEncoding}. Unpacks the block as necessary.
    */
-   private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock,
-       boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType,
-       DataBlockEncoding expectedDataBlockEncoding) throws IOException {
-     // Check cache for block. If found return.
-     if (cacheConf.isBlockCacheEnabled()) {
-       BlockCache cache = cacheConf.getBlockCache();
-       HFileBlock cachedBlock = (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock,
-         updateCacheMetrics);
-       if (cachedBlock != null) {
-         if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) {
-           HFileBlock compressedBlock = cachedBlock;
-           cachedBlock = compressedBlock.unpack(hfileContext, fsBlockReader);
-           // In case of compressed block after unpacking we can return the compressed block
+  private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock,
+      boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType,
+      DataBlockEncoding expectedDataBlockEncoding) throws IOException {
+    // Check cache for block. If found return.
+    if (cacheConf.getBlockCache().isPresent()) {
+      BlockCache cache = cacheConf.getBlockCache().get();
+      HFileBlock cachedBlock =
+          (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock, updateCacheMetrics);
+      if (cachedBlock != null) {
+        if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) {
+          HFileBlock compressedBlock = cachedBlock;
+          cachedBlock = compressedBlock.unpack(hfileContext, fsBlockReader);
+          // In case of compressed block after unpacking we can return the compressed block
           if (compressedBlock != cachedBlock) {
             cache.returnBlock(cacheKey, compressedBlock);
           }
         }
-         validateBlockType(cachedBlock, expectedBlockType);
-
-         if (expectedDataBlockEncoding == null) {
-           return cachedBlock;
-         }
-         DataBlockEncoding actualDataBlockEncoding =
-                 cachedBlock.getDataBlockEncoding();
-         // Block types other than data blocks always have
-         // DataBlockEncoding.NONE. To avoid false negative cache misses, only
-         // perform this check if cached block is a data block.
-         if (cachedBlock.getBlockType().isData() &&
-                 !actualDataBlockEncoding.equals(expectedDataBlockEncoding)) {
-           // This mismatch may happen if a Scanner, which is used for say a
-           // compaction, tries to read an encoded block from the block cache.
-           // The reverse might happen when an EncodedScanner tries to read
-           // un-encoded blocks which were cached earlier.
-           //
-           // Because returning a data block with an implicit BlockType mismatch
-           // will cause the requesting scanner to throw a disk read should be
-           // forced here. This will potentially cause a significant number of
-           // cache misses, so update so we should keep track of this as it might
-           // justify the work on a CompoundScanner.
-           if (!expectedDataBlockEncoding.equals(DataBlockEncoding.NONE) &&
-                   !actualDataBlockEncoding.equals(DataBlockEncoding.NONE)) {
-             // If the block is encoded but the encoding does not match the
-             // expected encoding it is likely the encoding was changed but the
-             // block was not yet evicted. Evictions on file close happen async
-             // so blocks with the old encoding still linger in cache for some
-             // period of time. This event should be rare as it only happens on
-             // schema definition change.
-             LOG.info("Evicting cached block with key " + cacheKey +
-                     " because of a data block encoding mismatch" +
-                     "; expected: " + expectedDataBlockEncoding +
-                     ", actual: " + actualDataBlockEncoding);
-             // This is an error scenario. so here we need to decrement the
-             // count.
-             cache.returnBlock(cacheKey, cachedBlock);
-             cache.evictBlock(cacheKey);
-           }
-           return null;
-         }
-         return cachedBlock;
-       }
-     }
-     return null;
-   }
+        validateBlockType(cachedBlock, expectedBlockType);
+
+        if (expectedDataBlockEncoding == null) {
+          return cachedBlock;
+        }
+        DataBlockEncoding actualDataBlockEncoding = cachedBlock.getDataBlockEncoding();
+        // Block types other than data blocks always have
+        // DataBlockEncoding.NONE. To avoid false negative cache misses, only
+        // perform this check if cached block is a data block.
+        if (cachedBlock.getBlockType().isData() &&
+            !actualDataBlockEncoding.equals(expectedDataBlockEncoding)) {
+          // This mismatch may happen if a Scanner, which is used for say a
+          // compaction, tries to read an encoded block from the block cache.
+          // The reverse might happen when an EncodedScanner tries to read
+          // un-encoded blocks which were cached earlier.
+          //
+          // Because returning a data block with an implicit BlockType mismatch
+          // will cause the requesting scanner to throw a disk read should be
+          // forced here. This will potentially cause a significant number of
+          // cache misses, so update so we should keep track of this as it might
+          // justify the work on a CompoundScanner.
+          if (!expectedDataBlockEncoding.equals(DataBlockEncoding.NONE) &&
+              !actualDataBlockEncoding.equals(DataBlockEncoding.NONE)) {
+            // If the block is encoded but the encoding does not match the
+            // expected encoding it is likely the encoding was changed but the
+            // block was not yet evicted. Evictions on file close happen async
+            // so blocks with the old encoding still linger in cache for some
+            // period of time. This event should be rare as it only happens on
+            // schema definition change.
+            LOG.info("Evicting cached block with key " + cacheKey +
+                " because of a data block encoding mismatch" + "; expected: " +
+                expectedDataBlockEncoding + ", actual: " + actualDataBlockEncoding);
+            // This is an error scenario. so here we need to decrement the
+            // count.
+            cache.returnBlock(cacheKey, cachedBlock);
+            cache.evictBlock(cacheKey);
+          }
+          return null;
+        }
+        return cachedBlock;
+      }
+    }
+    return null;
+  }
 
   /**
    * @param metaBlockName
@@ -1391,26 +1392,24 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         this.isPrimaryReplicaReader(), BlockType.META);
 
       cacheBlock &= cacheConf.shouldCacheBlockOnRead(BlockType.META.getCategory());
-      if (cacheConf.isBlockCacheEnabled()) {
-        HFileBlock cachedBlock = getCachedBlock(cacheKey, cacheBlock, false, true, true,
-          BlockType.META, null);
-        if (cachedBlock != null) {
-          assert cachedBlock.isUnpacked() : "Packed block leak.";
-          // Return a distinct 'shallow copy' of the block,
-          // so pos does not get messed by the scanner
-          return cachedBlock;
-        }
-        // Cache Miss, please load.
+      HFileBlock cachedBlock =
+          getCachedBlock(cacheKey, cacheBlock, false, true, true, BlockType.META, null);
+      if (cachedBlock != null) {
+        assert cachedBlock.isUnpacked() : "Packed block leak.";
+        // Return a distinct 'shallow copy' of the block,
+        // so pos does not get messed by the scanner
+        return cachedBlock;
       }
+      // Cache Miss, please load.
 
       HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, blockSize, true, false).
           unpack(hfileContext, fsBlockReader);
 
       // Cache the block
       if (cacheBlock) {
-        cacheConf.getBlockCache().cacheBlock(cacheKey, metaBlock, cacheConf.isInMemory());
+        cacheConf.getBlockCache()
+            .ifPresent(cache -> cache.cacheBlock(cacheKey, metaBlock, cacheConf.isInMemory()));
       }
-
       return metaBlock;
     }
   }
@@ -1491,11 +1490,13 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         BlockType.BlockCategory category = hfileBlock.getBlockType().getCategory();
 
         // Cache the block if necessary
-        if (cacheBlock && cacheConf.shouldCacheBlockOnRead(category)) {
-          cacheConf.getBlockCache().cacheBlock(cacheKey,
-            cacheConf.shouldCacheCompressed(category) ? hfileBlock : unpacked,
-            cacheConf.isInMemory());
-        }
+        cacheConf.getBlockCache().ifPresent(cache -> {
+          if (cacheBlock && cacheConf.shouldCacheBlockOnRead(category)) {
+            cache.cacheBlock(cacheKey,
+                cacheConf.shouldCacheCompressed(category) ? hfileBlock : unpacked,
+                cacheConf.isInMemory());
+          }
+        });
 
         if (updateCacheMetrics && hfileBlock.getBlockType().isData()) {
           HFile.DATABLOCK_READ_COUNT.increment();
@@ -1569,13 +1570,14 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
   @Override
   public void close(boolean evictOnClose) throws IOException {
     PrefetchExecutor.cancel(path);
-    if (evictOnClose && cacheConf.isBlockCacheEnabled()) {
-      int numEvicted = cacheConf.getBlockCache().evictBlocksByHfileName(name);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("On close, file=" + name + " evicted=" + numEvicted
-          + " block(s)");
+    cacheConf.getBlockCache().ifPresent(cache -> {
+      if (evictOnClose) {
+        int numEvicted = cache.evictBlocksByHfileName(name);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("On close, file=" + name + " evicted=" + numEvicted + " block(s)");
+        }
       }
-    }
+    });
     fsBlockReader.closeStreams();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index cfc3dd9..2726977 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -542,10 +542,11 @@ public class HFileWriterImpl implements HFile.Writer {
    *          the cache key.
    */
   private void doCacheOnWrite(long offset) {
-    HFileBlock cacheFormatBlock = blockWriter.getBlockForCaching(cacheConf);
-    cacheConf.getBlockCache().cacheBlock(
-        new BlockCacheKey(name, offset, true, cacheFormatBlock.getBlockType()),
-        cacheFormatBlock);
+    cacheConf.getBlockCache().ifPresent(cache -> {
+      HFileBlock cacheFormatBlock = blockWriter.getBlockForCaching(cacheConf);
+      cache.cacheBlock(new BlockCacheKey(name, offset, true, cacheFormatBlock.getBlockType()),
+          cacheFormatBlock);
+    });
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
index b8e4a0d..6caa975 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
@@ -29,11 +29,11 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.atomic.LongAdder;
 
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry;
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 
@@ -451,7 +451,7 @@ public final class BucketAllocator {
     BucketSizeInfo bsi = roundUpToBucketSizeInfo(blockSize);
     if (bsi == null) {
       throw new BucketAllocatorException("Allocation too big size=" + blockSize +
-        "; adjust BucketCache sizes " + CacheConfig.BUCKET_CACHE_BUCKETS_KEY +
+        "; adjust BucketCache sizes " + BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY +
         " to accomodate if size seems reasonable and you want it cached.");
     }
     long offset = bsi.allocateBlock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 7811d9b..5d84165 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -638,17 +638,18 @@ public class MergeTableRegionsProcedure
     final Configuration conf = env.getMasterConfiguration();
     final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
 
-    for (String family: regionFs.getFamilies()) {
+    for (String family : regionFs.getFamilies()) {
       final ColumnFamilyDescriptor hcd = htd.getColumnFamily(Bytes.toBytes(family));
       final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
 
       if (storeFiles != null && storeFiles.size() > 0) {
-        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
-        for (StoreFileInfo storeFileInfo: storeFiles) {
-          // Create reference file(s) of the region in mergedDir
-          regionFs.mergeStoreFile(mergedRegion, family, new HStoreFile(mfs.getFileSystem(),
-              storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true),
-            mergedDir);
+        for (StoreFileInfo storeFileInfo : storeFiles) {
+          // Create reference file(s) of the region in mergedDir.
+          // As this procedure is running on master, use CacheConfig.DISABLED means
+          // don't cache any block.
+          regionFs.mergeStoreFile(mergedRegion, family,
+              new HStoreFile(mfs.getFileSystem(), storeFileInfo, conf, CacheConfig.DISABLED,
+                  hcd.getBloomFilterType(), true), mergedDir);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index b66d91f..9894e25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -638,16 +638,17 @@ public class SplitTableRegionProcedure
 
     TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
     // Split each store file.
-    for (Map.Entry<String, Collection<StoreFileInfo>>e: files.entrySet()) {
-      byte [] familyName = Bytes.toBytes(e.getKey());
+    for (Map.Entry<String, Collection<StoreFileInfo>> e : files.entrySet()) {
+      byte[] familyName = Bytes.toBytes(e.getKey());
       final ColumnFamilyDescriptor hcd = htd.getColumnFamily(familyName);
       final Collection<StoreFileInfo> storeFiles = e.getValue();
       if (storeFiles != null && storeFiles.size() > 0) {
-        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
-        for (StoreFileInfo storeFileInfo: storeFiles) {
-          StoreFileSplitter sfs =
-              new StoreFileSplitter(regionFs, familyName, new HStoreFile(mfs.getFileSystem(),
-                  storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true));
+        for (StoreFileInfo storeFileInfo : storeFiles) {
+          // As this procedure is running on master, use CacheConfig.DISABLED means
+          // don't cache any block.
+          StoreFileSplitter sfs = new StoreFileSplitter(regionFs, familyName,
+              new HStoreFile(mfs.getFileSystem(), storeFileInfo, conf, CacheConfig.DISABLED,
+                  hcd.getBloomFilterType(), true));
           futures.add(threadPool.submit(sfs));
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
deleted file mode 100644
index 2305eba..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
+++ /dev/null
@@ -1,64 +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.mob;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-
-/**
- * The cache configuration for the mob.
- */
-@InterfaceAudience.Private
-public class MobCacheConfig extends CacheConfig {
-
-  private static MobFileCache mobFileCache;
-
-  public MobCacheConfig(Configuration conf, ColumnFamilyDescriptor family) {
-    super(conf, family);
-    instantiateMobFileCache(conf);
-  }
-
-  public MobCacheConfig(Configuration conf) {
-    super(conf);
-    instantiateMobFileCache(conf);
-  }
-
-  /**
-   * Instantiates the MobFileCache.
-   * @param conf The current configuration.
-   * @return The current instance of MobFileCache.
-   */
-  public static synchronized MobFileCache instantiateMobFileCache(Configuration conf) {
-    if (mobFileCache == null) {
-      mobFileCache = new MobFileCache(conf);
-    }
-    return mobFileCache;
-  }
-
-  /**
-   * Gets the MobFileCache.
-   * @return The MobFileCache.
-   */
-  public MobFileCache getMobFileCache() {
-    return mobFileCache;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
index b709d06..b987607 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
@@ -34,10 +34,11 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.util.IdLock;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.IdLock;
 
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -201,7 +202,7 @@ public class MobFileCache {
    * @return A opened mob file.
    * @throws IOException
    */
-  public MobFile openFile(FileSystem fs, Path path, MobCacheConfig cacheConf) throws IOException {
+  public MobFile openFile(FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
     if (!isCacheEnabled) {
       MobFile mobFile = MobFile.create(fs, path, conf, cacheConf);
       mobFile.open();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 740eb08..b657183 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
@@ -48,9 +48,9 @@ import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
-import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobFile;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.mob.MobStoreEngine;
 import org.apache.hadoop.hbase.mob.MobUtils;
@@ -80,7 +80,7 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public class HMobStore extends HStore {
   private static final Logger LOG = LoggerFactory.getLogger(HMobStore.class);
-  private MobCacheConfig mobCacheConfig;
+  private MobFileCache mobFileCache;
   private Path homePath;
   private Path mobFamilyPath;
   private AtomicLong cellsCountCompactedToMob = new AtomicLong();
@@ -107,7 +107,7 @@ public class HMobStore extends HStore {
       final Configuration confParam) throws IOException {
     super(region, family, confParam);
     this.family = family;
-    this.mobCacheConfig = (MobCacheConfig) cacheConf;
+    this.mobFileCache = region.getMobFileCache();
     this.homePath = MobUtils.getMobHome(conf);
     this.mobFamilyPath = MobUtils.getMobFamilyPath(conf, this.getTableName(),
         family.getNameAsString());
@@ -126,14 +126,6 @@ public class HMobStore extends HStore {
   }
 
   /**
-   * Creates the mob cache config.
-   */
-  @Override
-  protected void createCacheConf(ColumnFamilyDescriptor family) {
-    cacheConf = new MobCacheConfig(conf, family);
-  }
-
-  /**
    * Gets current config.
    */
   public Configuration getConfiguration() {
@@ -256,7 +248,7 @@ public class HMobStore extends HStore {
       long maxKeyCount, Compression.Algorithm compression,
       boolean isCompaction) throws IOException {
     return MobUtils.createWriter(conf, region.getFilesystem(), family,
-      new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, mobCacheConfig,
+      new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConf,
       cryptoContext, checksumType, bytesPerChecksum, blocksize, BloomType.NONE, isCompaction);
   }
 
@@ -291,7 +283,7 @@ public class HMobStore extends HStore {
   private void validateMobFile(Path path) throws IOException {
     HStoreFile storeFile = null;
     try {
-      storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
+      storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.cacheConf,
           BloomType.NONE, isPrimaryReplicaStore());
       storeFile.initReader();
     } catch (IOException e) {
@@ -398,11 +390,11 @@ public class HMobStore extends HStore {
       MobFile file = null;
       Path path = new Path(location, fileName);
       try {
-        file = mobCacheConfig.getMobFileCache().openFile(fs, path, mobCacheConfig);
+        file = mobFileCache.openFile(fs, path, cacheConf);
         return readPt != -1 ? file.readCell(search, cacheMobBlocks, readPt) : file.readCell(search,
           cacheMobBlocks);
       } catch (IOException e) {
-        mobCacheConfig.getMobFileCache().evictFile(fileName);
+        mobFileCache.evictFile(fileName);
         throwable = e;
         if ((e instanceof FileNotFoundException) ||
             (e.getCause() instanceof FileNotFoundException)) {
@@ -414,16 +406,16 @@ public class HMobStore extends HStore {
           throw e;
         }
       } catch (NullPointerException e) { // HDFS 1.x - DFSInputStream.getBlockAt()
-        mobCacheConfig.getMobFileCache().evictFile(fileName);
+        mobFileCache.evictFile(fileName);
         LOG.debug("Fail to read the cell", e);
         throwable = e;
       } catch (AssertionError e) { // assert in HDFS 1.x - DFSInputStream.getBlockAt()
-        mobCacheConfig.getMobFileCache().evictFile(fileName);
+        mobFileCache.evictFile(fileName);
         LOG.debug("Fail to read the cell", e);
         throwable = e;
       } finally {
         if (file != null) {
-          mobCacheConfig.getMobFileCache().closeFile(file);
+          mobFileCache.closeFile(file);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 4b6da53..21458c4 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
@@ -131,11 +131,13 @@ import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.CallerDisconnectedException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.RpcCall;
 import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
@@ -325,6 +327,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   final LongAdder compactionsQueued = new LongAdder();
   final LongAdder flushesQueued = new LongAdder();
 
+  private BlockCache blockCache;
+  private MobFileCache mobFileCache;
   private final WAL wal;
   private final HRegionFileSystem fs;
   protected final Configuration conf;
@@ -774,7 +778,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
       }
     }
+
     this.rsServices = rsServices;
+    if (this.rsServices != null) {
+      this.blockCache = rsServices.getBlockCache().orElse(null);
+      this.mobFileCache = rsServices.getMobFileCache().orElse(null);
+    }
+
     setHTableSpecificConf();
     this.scannerReadPoints = new ConcurrentHashMap<>();
 
@@ -1850,6 +1860,30 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return this.wal;
   }
 
+  public BlockCache getBlockCache() {
+    return this.blockCache;
+  }
+
+  /**
+   * Only used for unit test which doesn't start region server.
+   */
+  @VisibleForTesting
+  public void setBlockCache(BlockCache blockCache) {
+    this.blockCache = blockCache;
+  }
+
+  public MobFileCache getMobFileCache() {
+    return this.mobFileCache;
+  }
+
+  /**
+   * Only used for unit test which doesn't start region server.
+   */
+  @VisibleForTesting
+  public void setMobFileCache(MobFileCache mobFileCache) {
+    this.mobFileCache = mobFileCache;
+  }
+
   /**
    * @return split policy for this region.
    */
@@ -7062,7 +7096,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
     HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
     HRegion region = HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, null);
-    if (initialize) region.initialize(null);
+    if (initialize) {
+      region.initialize(null);
+    }
     return region;
   }
 
@@ -8130,7 +8166,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      53 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
+      55 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
       (15 * Bytes.SIZEOF_LONG) +
       3 * Bytes.SIZEOF_BOOLEAN);
 


[28/47] hbase git commit: HBASE-21629 draining_servers.rb is broken

Posted by zh...@apache.org.
HBASE-21629 draining_servers.rb is broken


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

Branch: refs/heads/HBASE-21512
Commit: 59f77de723849e4d330167f60e53e44b2763cafc
Parents: 97fd647
Author: Nihal Jain <ni...@gmail.com>
Authored: Sun Dec 23 14:29:53 2018 +0530
Committer: stack <st...@apache.org>
Committed: Sun Dec 23 20:48:43 2018 -0800

----------------------------------------------------------------------
 bin/draining_servers.rb | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59f77de7/bin/draining_servers.rb
----------------------------------------------------------------------
diff --git a/bin/draining_servers.rb b/bin/draining_servers.rb
index 0d29c19..a8e20f0 100644
--- a/bin/draining_servers.rb
+++ b/bin/draining_servers.rb
@@ -27,6 +27,7 @@ java_import org.apache.hadoop.hbase.HBaseConfiguration
 java_import org.apache.hadoop.hbase.client.ConnectionFactory
 java_import org.apache.hadoop.hbase.client.HBaseAdmin
 java_import org.apache.hadoop.hbase.zookeeper.ZKUtil
+java_import org.apache.hadoop.hbase.zookeeper.ZNodePaths
 java_import org.slf4j.LoggerFactory
 
 # Name of this script
@@ -86,11 +87,11 @@ def addServers(_options, hostOrServers)
   servers = getServerNames(hostOrServers, config)
 
   zkw = org.apache.hadoop.hbase.zookeeper.ZKWatcher.new(config, 'draining_servers', nil)
-  parentZnode = zkw.znodePaths.drainingZNode
 
   begin
+    parentZnode = zkw.getZNodePaths.drainingZNode
     for server in servers
-      node = ZKUtil.joinZNode(parentZnode, server)
+      node = ZNodePaths.joinZNode(parentZnode, server)
       ZKUtil.createAndFailSilent(zkw, node)
     end
   ensure
@@ -103,11 +104,11 @@ def removeServers(_options, hostOrServers)
   servers = getServerNames(hostOrServers, config)
 
   zkw = org.apache.hadoop.hbase.zookeeper.ZKWatcher.new(config, 'draining_servers', nil)
-  parentZnode = zkw.znodePaths.drainingZNode
 
   begin
+    parentZnode = zkw.getZNodePaths.drainingZNode
     for server in servers
-      node = ZKUtil.joinZNode(parentZnode, server)
+      node = ZNodePaths.joinZNode(parentZnode, server)
       ZKUtil.deleteNodeFailSilent(zkw, node)
     end
   ensure
@@ -120,10 +121,14 @@ def listServers(_options)
   config = HBaseConfiguration.create
 
   zkw = org.apache.hadoop.hbase.zookeeper.ZKWatcher.new(config, 'draining_servers', nil)
-  parentZnode = zkw.znodePaths.drainingZNode
 
-  servers = ZKUtil.listChildrenNoWatch(zkw, parentZnode)
-  servers.each { |server| puts server }
+  begin
+    parentZnode = zkw.getZNodePaths.drainingZNode
+    servers = ZKUtil.listChildrenNoWatch(zkw, parentZnode)
+    servers.each { |server| puts server }
+  ensure
+    zkw.close
+  end
 end
 
 hostOrServers = ARGV[1..ARGV.size]


[27/47] hbase git commit: HBASE-21635 Use maven enforcer to ban imports from illegal packages

Posted by zh...@apache.org.
HBASE-21635 Use maven enforcer to ban imports from illegal packages


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

Branch: refs/heads/HBASE-21512
Commit: 97fd647de20e6c8df0cd6df248ec1365abc37378
Parents: 7c0a3cc
Author: zhangduo <zh...@apache.org>
Authored: Sun Dec 23 18:25:42 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Mon Dec 24 11:12:25 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/master/DeadServer.java  | 17 ++--
 .../TestBalancerStatusTagInJMXMetrics.java      | 32 ++++----
 pom.xml                                         | 83 +++++++++++++++++++-
 3 files changed, 107 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/97fd647d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
index 4183201..0584792 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
@@ -18,15 +18,6 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.base.Preconditions;
-
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -37,6 +28,14 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/97fd647d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java
index 9f56621..d23436d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java
@@ -1,22 +1,25 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.balancer;
 
 import static org.junit.Assert.assertEquals;
 
 import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -31,6 +34,8 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Category({ MediumTests.class })
 public class TestBalancerStatusTagInJMXMetrics extends BalancerTestBase {
@@ -39,7 +44,7 @@ public class TestBalancerStatusTagInJMXMetrics extends BalancerTestBase {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestBalancerStatusTagInJMXMetrics.class);
 
-  private static final Log LOG = LogFactory.getLog(TestBalancerStatusTagInJMXMetrics.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestBalancerStatusTagInJMXMetrics.class);
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static int connectorPort = 61120;
   private static HMaster master;
@@ -51,7 +56,6 @@ public class TestBalancerStatusTagInJMXMetrics extends BalancerTestBase {
    */
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
-
     conf = UTIL.getConfiguration();
     Random rand = new Random();
     for (int i = 0; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/97fd647d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ebd97f7..a455b00 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1077,6 +1077,11 @@
             <artifactId>extra-enforcer-rules</artifactId>
             <version>${extra.enforcer.version}</version>
           </dependency>
+          <dependency>
+            <groupId>de.skuzzle.enforcer</groupId>
+            <artifactId>restrict-imports-enforcer-rule</artifactId>
+            <version>${restrict-imports.enforcer.version}</version>
+          </dependency>
         </dependencies>
         <!-- version set by parent -->
         <executions>
@@ -1188,6 +1193,79 @@
               <skip>${skip.license.check}</skip>
             </configuration>
           </execution>
+          <execution>
+            <id>banned-illegal-imports</id>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <commentLineBufferSize>512</commentLineBufferSize>
+                  <reason>Use SLF4j for logging</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.commons.logging.Log</bannedImport>
+                    <bannedImport>org.apache.commons.logging.LogFactory</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <commentLineBufferSize>512</commentLineBufferSize>
+                  <reason>Use shaded version in hbase-thirdparty</reason>
+                  <bannedImports>
+                    <bannedImport>com.google.common.**</bannedImport>
+                    <bannedImport>io.netty.**</bannedImport>
+                    <bannedImport>org.apache.commons.cli.**</bannedImport>
+                    <bannedImport>org.apache.commons.collections.**</bannedImport>
+                    <bannedImport>org.apache.commons.collections4.**</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <commentLineBufferSize>512</commentLineBufferSize>
+                  <reason>Do not use shaded classes from other dependencies</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.curator.shaded.**</bannedImport>
+                    <bannedImport>org.apache.htrace.shaded.**</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <commentLineBufferSize>512</commentLineBufferSize>
+                  <reason>Use shaded gson in hbase-thirdparty</reason>
+                  <bannedImports>
+                    <bannedImport>org.codehaus.jackson.**</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <commentLineBufferSize>512</commentLineBufferSize>
+                  <reason>Use commons lang 3</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.commons.lang.**</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <commentLineBufferSize>512</commentLineBufferSize>
+                  <reason>Use yetus IA and IS annotations</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.hadoop.classificatio.**</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <commentLineBufferSize>512</commentLineBufferSize>
+                  <reason>Do not use htrace v3</reason>
+                  <bannedImports>
+                    <bannedImport>org.htrace.**</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+              </rules>
+            </configuration>
+          </execution>
         </executions>
       </plugin>
       <!-- parent-module only plugins -->
@@ -1599,8 +1677,9 @@
     <!-- Surefire argLine defaults to Linux, cygwin argLine is used in the os.windows profile -->
     <argLine>${hbase-surefire.argLine}</argLine>
     <jacoco.version>0.7.5.201505241946</jacoco.version>
-    <extra.enforcer.version>1.0-beta-6</extra.enforcer.version>
-    <enforcer.version>3.0.0-M1</enforcer.version>
+    <extra.enforcer.version>1.0-beta-9</extra.enforcer.version>
+    <enforcer.version>3.0.0-M2</enforcer.version>
+    <restrict-imports.enforcer.version>0.14.0</restrict-imports.enforcer.version>
     <!-- Location of test resources -->
     <test.build.classes>${project.build.directory}/test-classes</test.build.classes>
     <maven.build.timestamp.format>yyyy-MM-dd'T'HH:mm:ss'Z'</maven.build.timestamp.format>


[31/47] hbase git commit: HBASE-21640 Remove the TODO when increment zero

Posted by zh...@apache.org.
HBASE-21640 Remove the TODO when increment zero


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

Branch: refs/heads/HBASE-21512
Commit: 4281cb3b9574333fab0e7c028c9c0d7e5b320c73
Parents: 44dec60
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 25 17:42:38 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Wed Dec 26 21:47:44 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 21 ++++----------------
 .../hbase/regionserver/wal/TestDurability.java  |  9 ++++-----
 2 files changed, 8 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4281cb3b/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 21458c4..dc0fa22 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
@@ -7963,8 +7963,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   /**
    * Reckon the Cells to apply to WAL, memstore, and to return to the Client; these Sets are not
-   * always the same dependent on whether to write WAL or if the amount to increment is zero (in
-   * this case we write back nothing, just return latest Cell value to the client).
+   * always the same dependent on whether to write WAL.
    *
    * @param results Fill in here what goes back to the Client if it is non-null (if null, client
    *  doesn't want results).
@@ -8006,9 +8005,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param op Whether Increment or Append
    * @param mutation The encompassing Mutation object
    * @param deltas Changes to apply to this Store; either increment amount or data to append
-   * @param results In here we accumulate all the Cells we are to return to the client; this List
-   *  can be larger than what we return in case where delta is zero; i.e. don't write
-   *  out new values, just return current value. If null, client doesn't want results returned.
+   * @param results In here we accumulate all the Cells we are to return to the client. If null,
+   *                client doesn't want results returned.
    * @return Resulting Cells after <code>deltas</code> have been applied to current
    *  values. Side effect is our filling out of the <code>results</code> List.
    */
@@ -8036,33 +8034,25 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     for (int i = 0; i < deltas.size(); i++) {
       Cell delta = deltas.get(i);
       Cell currentValue = null;
-      boolean firstWrite = false;
       if (currentValuesIndex < currentValues.size() &&
           CellUtil.matchingQualifier(currentValues.get(currentValuesIndex), delta)) {
         currentValue = currentValues.get(currentValuesIndex);
         if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) {
           currentValuesIndex++;
         }
-      } else {
-        firstWrite = true;
       }
       // Switch on whether this an increment or an append building the new Cell to apply.
       Cell newCell = null;
       MutationType mutationType = null;
-      boolean apply = true;
       switch (op) {
         case INCREMENT:
           mutationType = MutationType.INCREMENT;
-          // If delta amount to apply is 0, don't write WAL or MemStore.
           long deltaAmount = getLongValue(delta);
-          // TODO: Does zero value mean reset Cell? For example, the ttl.
-          apply = deltaAmount != 0;
           final long newValue = currentValue == null ? deltaAmount : getLongValue(currentValue) + deltaAmount;
           newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, (oldCell) -> Bytes.toBytes(newValue));
           break;
         case APPEND:
           mutationType = MutationType.APPEND;
-          // Always apply Append. TODO: Does empty delta value mean reset Cell? It seems to.
           newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, (oldCell) ->
             ByteBuffer.wrap(new byte[delta.getValueLength() + oldCell.getValueLength()])
                     .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength())
@@ -8078,10 +8068,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         newCell =
             coprocessorHost.postMutationBeforeWAL(mutationType, mutation, currentValue, newCell);
       }
-      // If apply, we need to update memstore/WAL with new value; add it toApply.
-      if (apply || firstWrite) {
-        toApply.add(newCell);
-      }
+      toApply.add(newCell);
       // Add to results to get returned to the Client. If null, cilent does not want results.
       if (results != null) {
         results.add(newCell);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4281cb3b/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 4effa6d..320bcbf 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
@@ -208,14 +208,13 @@ public class TestDurability {
     assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1)));
     verifyWALCount(wals, wal, 2);
 
-    // col1: amount = 0, 0 write back to WAL
+    // col1: amount = 0, 1 write back to WAL
     inc1 = new Increment(row1);
     inc1.addColumn(FAMILY, col1, 0);
     res = region.increment(inc1);
     assertEquals(1, res.size());
     assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1)));
-    verifyWALCount(wals, wal, 2);
-
+    verifyWALCount(wals, wal, 3);
     // col1: amount = 0, col2: amount = 0, col3: amount = 0
     // 1 write back to WAL
     inc1 = new Increment(row1);
@@ -227,7 +226,7 @@ public class TestDurability {
     assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1)));
     assertEquals(0, Bytes.toLong(res.getValue(FAMILY, col2)));
     assertEquals(0, Bytes.toLong(res.getValue(FAMILY, col3)));
-    verifyWALCount(wals, wal, 3);
+    verifyWALCount(wals, wal, 4);
 
     // col1: amount = 5, col2: amount = 4, col3: amount = 3
     // 1 write back to WAL
@@ -240,7 +239,7 @@ public class TestDurability {
     assertEquals(6, Bytes.toLong(res.getValue(FAMILY, col1)));
     assertEquals(4, Bytes.toLong(res.getValue(FAMILY, col2)));
     assertEquals(3, Bytes.toLong(res.getValue(FAMILY, col3)));
-    verifyWALCount(wals, wal, 4);
+    verifyWALCount(wals, wal, 5);
   }
 
   /**


[39/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
new file mode 100644
index 0000000..03cb2f6
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
@@ -0,0 +1,2519 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+public class TColumnFamilyDescriptor implements org.apache.thrift.TBase<TColumnFamilyDescriptor, TColumnFamilyDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnFamilyDescriptor> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField ATTRIBUTES_FIELD_DESC = new org.apache.thrift.protocol.TField("attributes", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField BLOCK_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("blockSize", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField BLOOMN_FILTER_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("bloomnFilterType", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField COMPRESSION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("compressionType", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField DFS_REPLICATION_FIELD_DESC = new org.apache.thrift.protocol.TField("dfsReplication", org.apache.thrift.protocol.TType.I16, (short)7);
+  private static final org.apache.thrift.protocol.TField DATA_BLOCK_ENCODING_FIELD_DESC = new org.apache.thrift.protocol.TField("dataBlockEncoding", org.apache.thrift.protocol.TType.I32, (short)8);
+  private static final org.apache.thrift.protocol.TField KEEP_DELETED_CELLS_FIELD_DESC = new org.apache.thrift.protocol.TField("keepDeletedCells", org.apache.thrift.protocol.TType.I32, (short)9);
+  private static final org.apache.thrift.protocol.TField MAX_VERSIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxVersions", org.apache.thrift.protocol.TType.I32, (short)10);
+  private static final org.apache.thrift.protocol.TField MIN_VERSIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("minVersions", org.apache.thrift.protocol.TType.I32, (short)11);
+  private static final org.apache.thrift.protocol.TField SCOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("scope", org.apache.thrift.protocol.TType.I32, (short)12);
+  private static final org.apache.thrift.protocol.TField TIME_TO_LIVE_FIELD_DESC = new org.apache.thrift.protocol.TField("timeToLive", org.apache.thrift.protocol.TType.I32, (short)13);
+  private static final org.apache.thrift.protocol.TField BLOCK_CACHE_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("blockCacheEnabled", org.apache.thrift.protocol.TType.BOOL, (short)14);
+  private static final org.apache.thrift.protocol.TField CACHE_BLOOMS_ON_WRITE_FIELD_DESC = new org.apache.thrift.protocol.TField("cacheBloomsOnWrite", org.apache.thrift.protocol.TType.BOOL, (short)15);
+  private static final org.apache.thrift.protocol.TField CACHE_DATA_ON_WRITE_FIELD_DESC = new org.apache.thrift.protocol.TField("cacheDataOnWrite", org.apache.thrift.protocol.TType.BOOL, (short)16);
+  private static final org.apache.thrift.protocol.TField CACHE_INDEXES_ON_WRITE_FIELD_DESC = new org.apache.thrift.protocol.TField("cacheIndexesOnWrite", org.apache.thrift.protocol.TType.BOOL, (short)17);
+  private static final org.apache.thrift.protocol.TField COMPRESS_TAGS_FIELD_DESC = new org.apache.thrift.protocol.TField("compressTags", org.apache.thrift.protocol.TType.BOOL, (short)18);
+  private static final org.apache.thrift.protocol.TField EVICT_BLOCKS_ON_CLOSE_FIELD_DESC = new org.apache.thrift.protocol.TField("evictBlocksOnClose", org.apache.thrift.protocol.TType.BOOL, (short)19);
+  private static final org.apache.thrift.protocol.TField IN_MEMORY_FIELD_DESC = new org.apache.thrift.protocol.TField("inMemory", org.apache.thrift.protocol.TType.BOOL, (short)20);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TColumnFamilyDescriptorStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TColumnFamilyDescriptorTupleSchemeFactory());
+  }
+
+  public ByteBuffer name; // required
+  public Map<ByteBuffer,ByteBuffer> attributes; // optional
+  public Map<String,String> configuration; // optional
+  public int blockSize; // optional
+  /**
+   * 
+   * @see TBloomFilterType
+   */
+  public TBloomFilterType bloomnFilterType; // optional
+  /**
+   * 
+   * @see TCompressionAlgorithm
+   */
+  public TCompressionAlgorithm compressionType; // optional
+  public short dfsReplication; // optional
+  /**
+   * 
+   * @see TDataBlockEncoding
+   */
+  public TDataBlockEncoding dataBlockEncoding; // optional
+  /**
+   * 
+   * @see TKeepDeletedCells
+   */
+  public TKeepDeletedCells keepDeletedCells; // optional
+  public int maxVersions; // optional
+  public int minVersions; // optional
+  public int scope; // optional
+  public int timeToLive; // optional
+  public boolean blockCacheEnabled; // optional
+  public boolean cacheBloomsOnWrite; // optional
+  public boolean cacheDataOnWrite; // optional
+  public boolean cacheIndexesOnWrite; // optional
+  public boolean compressTags; // optional
+  public boolean evictBlocksOnClose; // optional
+  public boolean inMemory; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    ATTRIBUTES((short)2, "attributes"),
+    CONFIGURATION((short)3, "configuration"),
+    BLOCK_SIZE((short)4, "blockSize"),
+    /**
+     * 
+     * @see TBloomFilterType
+     */
+    BLOOMN_FILTER_TYPE((short)5, "bloomnFilterType"),
+    /**
+     * 
+     * @see TCompressionAlgorithm
+     */
+    COMPRESSION_TYPE((short)6, "compressionType"),
+    DFS_REPLICATION((short)7, "dfsReplication"),
+    /**
+     * 
+     * @see TDataBlockEncoding
+     */
+    DATA_BLOCK_ENCODING((short)8, "dataBlockEncoding"),
+    /**
+     * 
+     * @see TKeepDeletedCells
+     */
+    KEEP_DELETED_CELLS((short)9, "keepDeletedCells"),
+    MAX_VERSIONS((short)10, "maxVersions"),
+    MIN_VERSIONS((short)11, "minVersions"),
+    SCOPE((short)12, "scope"),
+    TIME_TO_LIVE((short)13, "timeToLive"),
+    BLOCK_CACHE_ENABLED((short)14, "blockCacheEnabled"),
+    CACHE_BLOOMS_ON_WRITE((short)15, "cacheBloomsOnWrite"),
+    CACHE_DATA_ON_WRITE((short)16, "cacheDataOnWrite"),
+    CACHE_INDEXES_ON_WRITE((short)17, "cacheIndexesOnWrite"),
+    COMPRESS_TAGS((short)18, "compressTags"),
+    EVICT_BLOCKS_ON_CLOSE((short)19, "evictBlocksOnClose"),
+    IN_MEMORY((short)20, "inMemory");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // ATTRIBUTES
+          return ATTRIBUTES;
+        case 3: // CONFIGURATION
+          return CONFIGURATION;
+        case 4: // BLOCK_SIZE
+          return BLOCK_SIZE;
+        case 5: // BLOOMN_FILTER_TYPE
+          return BLOOMN_FILTER_TYPE;
+        case 6: // COMPRESSION_TYPE
+          return COMPRESSION_TYPE;
+        case 7: // DFS_REPLICATION
+          return DFS_REPLICATION;
+        case 8: // DATA_BLOCK_ENCODING
+          return DATA_BLOCK_ENCODING;
+        case 9: // KEEP_DELETED_CELLS
+          return KEEP_DELETED_CELLS;
+        case 10: // MAX_VERSIONS
+          return MAX_VERSIONS;
+        case 11: // MIN_VERSIONS
+          return MIN_VERSIONS;
+        case 12: // SCOPE
+          return SCOPE;
+        case 13: // TIME_TO_LIVE
+          return TIME_TO_LIVE;
+        case 14: // BLOCK_CACHE_ENABLED
+          return BLOCK_CACHE_ENABLED;
+        case 15: // CACHE_BLOOMS_ON_WRITE
+          return CACHE_BLOOMS_ON_WRITE;
+        case 16: // CACHE_DATA_ON_WRITE
+          return CACHE_DATA_ON_WRITE;
+        case 17: // CACHE_INDEXES_ON_WRITE
+          return CACHE_INDEXES_ON_WRITE;
+        case 18: // COMPRESS_TAGS
+          return COMPRESS_TAGS;
+        case 19: // EVICT_BLOCKS_ON_CLOSE
+          return EVICT_BLOCKS_ON_CLOSE;
+        case 20: // IN_MEMORY
+          return IN_MEMORY;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __BLOCKSIZE_ISSET_ID = 0;
+  private static final int __DFSREPLICATION_ISSET_ID = 1;
+  private static final int __MAXVERSIONS_ISSET_ID = 2;
+  private static final int __MINVERSIONS_ISSET_ID = 3;
+  private static final int __SCOPE_ISSET_ID = 4;
+  private static final int __TIMETOLIVE_ISSET_ID = 5;
+  private static final int __BLOCKCACHEENABLED_ISSET_ID = 6;
+  private static final int __CACHEBLOOMSONWRITE_ISSET_ID = 7;
+  private static final int __CACHEDATAONWRITE_ISSET_ID = 8;
+  private static final int __CACHEINDEXESONWRITE_ISSET_ID = 9;
+  private static final int __COMPRESSTAGS_ISSET_ID = 10;
+  private static final int __EVICTBLOCKSONCLOSE_ISSET_ID = 11;
+  private static final int __INMEMORY_ISSET_ID = 12;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.CONFIGURATION,_Fields.BLOCK_SIZE,_Fields.BLOOMN_FILTER_TYPE,_Fields.COMPRESSION_TYPE,_Fields.DFS_REPLICATION,_Fields.DATA_BLOCK_ENCODING,_Fields.KEEP_DELETED_CELLS,_Fields.MAX_VERSIONS,_Fields.MIN_VERSIONS,_Fields.SCOPE,_Fields.TIME_TO_LIVE,_Fields.BLOCK_CACHE_ENABLED,_Fields.CACHE_BLOOMS_ON_WRITE,_Fields.CACHE_DATA_ON_WRITE,_Fields.CACHE_INDEXES_ON_WRITE,_Fields.COMPRESS_TAGS,_Fields.EVICT_BLOCKS_ON_CLOSE,_Fields.IN_MEMORY};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.ATTRIBUTES, new org.apache.thrift.meta_data.FieldMetaData("attributes", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
+    tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.BLOCK_SIZE, new org.apache.thrift.meta_data.FieldMetaData("blockSize", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.BLOOMN_FILTER_TYPE, new org.apache.thrift.meta_data.FieldMetaData("bloomnFilterType", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TBloomFilterType.class)));
+    tmpMap.put(_Fields.COMPRESSION_TYPE, new org.apache.thrift.meta_data.FieldMetaData("compressionType", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TCompressionAlgorithm.class)));
+    tmpMap.put(_Fields.DFS_REPLICATION, new org.apache.thrift.meta_data.FieldMetaData("dfsReplication", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16)));
+    tmpMap.put(_Fields.DATA_BLOCK_ENCODING, new org.apache.thrift.meta_data.FieldMetaData("dataBlockEncoding", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDataBlockEncoding.class)));
+    tmpMap.put(_Fields.KEEP_DELETED_CELLS, new org.apache.thrift.meta_data.FieldMetaData("keepDeletedCells", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TKeepDeletedCells.class)));
+    tmpMap.put(_Fields.MAX_VERSIONS, new org.apache.thrift.meta_data.FieldMetaData("maxVersions", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.MIN_VERSIONS, new org.apache.thrift.meta_data.FieldMetaData("minVersions", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.SCOPE, new org.apache.thrift.meta_data.FieldMetaData("scope", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.TIME_TO_LIVE, new org.apache.thrift.meta_data.FieldMetaData("timeToLive", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.BLOCK_CACHE_ENABLED, new org.apache.thrift.meta_data.FieldMetaData("blockCacheEnabled", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CACHE_BLOOMS_ON_WRITE, new org.apache.thrift.meta_data.FieldMetaData("cacheBloomsOnWrite", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CACHE_DATA_ON_WRITE, new org.apache.thrift.meta_data.FieldMetaData("cacheDataOnWrite", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CACHE_INDEXES_ON_WRITE, new org.apache.thrift.meta_data.FieldMetaData("cacheIndexesOnWrite", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.COMPRESS_TAGS, new org.apache.thrift.meta_data.FieldMetaData("compressTags", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.EVICT_BLOCKS_ON_CLOSE, new org.apache.thrift.meta_data.FieldMetaData("evictBlocksOnClose", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.IN_MEMORY, new org.apache.thrift.meta_data.FieldMetaData("inMemory", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnFamilyDescriptor.class, metaDataMap);
+  }
+
+  public TColumnFamilyDescriptor() {
+  }
+
+  public TColumnFamilyDescriptor(
+    ByteBuffer name)
+  {
+    this();
+    this.name = org.apache.thrift.TBaseHelper.copyBinary(name);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TColumnFamilyDescriptor(TColumnFamilyDescriptor other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetName()) {
+      this.name = org.apache.thrift.TBaseHelper.copyBinary(other.name);
+    }
+    if (other.isSetAttributes()) {
+      Map<ByteBuffer,ByteBuffer> __this__attributes = new HashMap<ByteBuffer,ByteBuffer>(other.attributes);
+      this.attributes = __this__attributes;
+    }
+    if (other.isSetConfiguration()) {
+      Map<String,String> __this__configuration = new HashMap<String,String>(other.configuration);
+      this.configuration = __this__configuration;
+    }
+    this.blockSize = other.blockSize;
+    if (other.isSetBloomnFilterType()) {
+      this.bloomnFilterType = other.bloomnFilterType;
+    }
+    if (other.isSetCompressionType()) {
+      this.compressionType = other.compressionType;
+    }
+    this.dfsReplication = other.dfsReplication;
+    if (other.isSetDataBlockEncoding()) {
+      this.dataBlockEncoding = other.dataBlockEncoding;
+    }
+    if (other.isSetKeepDeletedCells()) {
+      this.keepDeletedCells = other.keepDeletedCells;
+    }
+    this.maxVersions = other.maxVersions;
+    this.minVersions = other.minVersions;
+    this.scope = other.scope;
+    this.timeToLive = other.timeToLive;
+    this.blockCacheEnabled = other.blockCacheEnabled;
+    this.cacheBloomsOnWrite = other.cacheBloomsOnWrite;
+    this.cacheDataOnWrite = other.cacheDataOnWrite;
+    this.cacheIndexesOnWrite = other.cacheIndexesOnWrite;
+    this.compressTags = other.compressTags;
+    this.evictBlocksOnClose = other.evictBlocksOnClose;
+    this.inMemory = other.inMemory;
+  }
+
+  public TColumnFamilyDescriptor deepCopy() {
+    return new TColumnFamilyDescriptor(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.attributes = null;
+    this.configuration = null;
+    setBlockSizeIsSet(false);
+    this.blockSize = 0;
+    this.bloomnFilterType = null;
+    this.compressionType = null;
+    setDfsReplicationIsSet(false);
+    this.dfsReplication = 0;
+    this.dataBlockEncoding = null;
+    this.keepDeletedCells = null;
+    setMaxVersionsIsSet(false);
+    this.maxVersions = 0;
+    setMinVersionsIsSet(false);
+    this.minVersions = 0;
+    setScopeIsSet(false);
+    this.scope = 0;
+    setTimeToLiveIsSet(false);
+    this.timeToLive = 0;
+    setBlockCacheEnabledIsSet(false);
+    this.blockCacheEnabled = false;
+    setCacheBloomsOnWriteIsSet(false);
+    this.cacheBloomsOnWrite = false;
+    setCacheDataOnWriteIsSet(false);
+    this.cacheDataOnWrite = false;
+    setCacheIndexesOnWriteIsSet(false);
+    this.cacheIndexesOnWrite = false;
+    setCompressTagsIsSet(false);
+    this.compressTags = false;
+    setEvictBlocksOnCloseIsSet(false);
+    this.evictBlocksOnClose = false;
+    setInMemoryIsSet(false);
+    this.inMemory = false;
+  }
+
+  public byte[] getName() {
+    setName(org.apache.thrift.TBaseHelper.rightSize(name));
+    return name == null ? null : name.array();
+  }
+
+  public ByteBuffer bufferForName() {
+    return org.apache.thrift.TBaseHelper.copyBinary(name);
+  }
+
+  public TColumnFamilyDescriptor setName(byte[] name) {
+    this.name = name == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(name, name.length));
+    return this;
+  }
+
+  public TColumnFamilyDescriptor setName(ByteBuffer name) {
+    this.name = org.apache.thrift.TBaseHelper.copyBinary(name);
+    return this;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int getAttributesSize() {
+    return (this.attributes == null) ? 0 : this.attributes.size();
+  }
+
+  public void putToAttributes(ByteBuffer key, ByteBuffer val) {
+    if (this.attributes == null) {
+      this.attributes = new HashMap<ByteBuffer,ByteBuffer>();
+    }
+    this.attributes.put(key, val);
+  }
+
+  public Map<ByteBuffer,ByteBuffer> getAttributes() {
+    return this.attributes;
+  }
+
+  public TColumnFamilyDescriptor setAttributes(Map<ByteBuffer,ByteBuffer> attributes) {
+    this.attributes = attributes;
+    return this;
+  }
+
+  public void unsetAttributes() {
+    this.attributes = null;
+  }
+
+  /** Returns true if field attributes is set (has been assigned a value) and false otherwise */
+  public boolean isSetAttributes() {
+    return this.attributes != null;
+  }
+
+  public void setAttributesIsSet(boolean value) {
+    if (!value) {
+      this.attributes = null;
+    }
+  }
+
+  public int getConfigurationSize() {
+    return (this.configuration == null) ? 0 : this.configuration.size();
+  }
+
+  public void putToConfiguration(String key, String val) {
+    if (this.configuration == null) {
+      this.configuration = new HashMap<String,String>();
+    }
+    this.configuration.put(key, val);
+  }
+
+  public Map<String,String> getConfiguration() {
+    return this.configuration;
+  }
+
+  public TColumnFamilyDescriptor setConfiguration(Map<String,String> configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  public void unsetConfiguration() {
+    this.configuration = null;
+  }
+
+  /** Returns true if field configuration is set (has been assigned a value) and false otherwise */
+  public boolean isSetConfiguration() {
+    return this.configuration != null;
+  }
+
+  public void setConfigurationIsSet(boolean value) {
+    if (!value) {
+      this.configuration = null;
+    }
+  }
+
+  public int getBlockSize() {
+    return this.blockSize;
+  }
+
+  public TColumnFamilyDescriptor setBlockSize(int blockSize) {
+    this.blockSize = blockSize;
+    setBlockSizeIsSet(true);
+    return this;
+  }
+
+  public void unsetBlockSize() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BLOCKSIZE_ISSET_ID);
+  }
+
+  /** Returns true if field blockSize is set (has been assigned a value) and false otherwise */
+  public boolean isSetBlockSize() {
+    return EncodingUtils.testBit(__isset_bitfield, __BLOCKSIZE_ISSET_ID);
+  }
+
+  public void setBlockSizeIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BLOCKSIZE_ISSET_ID, value);
+  }
+
+  /**
+   * 
+   * @see TBloomFilterType
+   */
+  public TBloomFilterType getBloomnFilterType() {
+    return this.bloomnFilterType;
+  }
+
+  /**
+   * 
+   * @see TBloomFilterType
+   */
+  public TColumnFamilyDescriptor setBloomnFilterType(TBloomFilterType bloomnFilterType) {
+    this.bloomnFilterType = bloomnFilterType;
+    return this;
+  }
+
+  public void unsetBloomnFilterType() {
+    this.bloomnFilterType = null;
+  }
+
+  /** Returns true if field bloomnFilterType is set (has been assigned a value) and false otherwise */
+  public boolean isSetBloomnFilterType() {
+    return this.bloomnFilterType != null;
+  }
+
+  public void setBloomnFilterTypeIsSet(boolean value) {
+    if (!value) {
+      this.bloomnFilterType = null;
+    }
+  }
+
+  /**
+   * 
+   * @see TCompressionAlgorithm
+   */
+  public TCompressionAlgorithm getCompressionType() {
+    return this.compressionType;
+  }
+
+  /**
+   * 
+   * @see TCompressionAlgorithm
+   */
+  public TColumnFamilyDescriptor setCompressionType(TCompressionAlgorithm compressionType) {
+    this.compressionType = compressionType;
+    return this;
+  }
+
+  public void unsetCompressionType() {
+    this.compressionType = null;
+  }
+
+  /** Returns true if field compressionType is set (has been assigned a value) and false otherwise */
+  public boolean isSetCompressionType() {
+    return this.compressionType != null;
+  }
+
+  public void setCompressionTypeIsSet(boolean value) {
+    if (!value) {
+      this.compressionType = null;
+    }
+  }
+
+  public short getDfsReplication() {
+    return this.dfsReplication;
+  }
+
+  public TColumnFamilyDescriptor setDfsReplication(short dfsReplication) {
+    this.dfsReplication = dfsReplication;
+    setDfsReplicationIsSet(true);
+    return this;
+  }
+
+  public void unsetDfsReplication() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DFSREPLICATION_ISSET_ID);
+  }
+
+  /** Returns true if field dfsReplication is set (has been assigned a value) and false otherwise */
+  public boolean isSetDfsReplication() {
+    return EncodingUtils.testBit(__isset_bitfield, __DFSREPLICATION_ISSET_ID);
+  }
+
+  public void setDfsReplicationIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DFSREPLICATION_ISSET_ID, value);
+  }
+
+  /**
+   * 
+   * @see TDataBlockEncoding
+   */
+  public TDataBlockEncoding getDataBlockEncoding() {
+    return this.dataBlockEncoding;
+  }
+
+  /**
+   * 
+   * @see TDataBlockEncoding
+   */
+  public TColumnFamilyDescriptor setDataBlockEncoding(TDataBlockEncoding dataBlockEncoding) {
+    this.dataBlockEncoding = dataBlockEncoding;
+    return this;
+  }
+
+  public void unsetDataBlockEncoding() {
+    this.dataBlockEncoding = null;
+  }
+
+  /** Returns true if field dataBlockEncoding is set (has been assigned a value) and false otherwise */
+  public boolean isSetDataBlockEncoding() {
+    return this.dataBlockEncoding != null;
+  }
+
+  public void setDataBlockEncodingIsSet(boolean value) {
+    if (!value) {
+      this.dataBlockEncoding = null;
+    }
+  }
+
+  /**
+   * 
+   * @see TKeepDeletedCells
+   */
+  public TKeepDeletedCells getKeepDeletedCells() {
+    return this.keepDeletedCells;
+  }
+
+  /**
+   * 
+   * @see TKeepDeletedCells
+   */
+  public TColumnFamilyDescriptor setKeepDeletedCells(TKeepDeletedCells keepDeletedCells) {
+    this.keepDeletedCells = keepDeletedCells;
+    return this;
+  }
+
+  public void unsetKeepDeletedCells() {
+    this.keepDeletedCells = null;
+  }
+
+  /** Returns true if field keepDeletedCells is set (has been assigned a value) and false otherwise */
+  public boolean isSetKeepDeletedCells() {
+    return this.keepDeletedCells != null;
+  }
+
+  public void setKeepDeletedCellsIsSet(boolean value) {
+    if (!value) {
+      this.keepDeletedCells = null;
+    }
+  }
+
+  public int getMaxVersions() {
+    return this.maxVersions;
+  }
+
+  public TColumnFamilyDescriptor setMaxVersions(int maxVersions) {
+    this.maxVersions = maxVersions;
+    setMaxVersionsIsSet(true);
+    return this;
+  }
+
+  public void unsetMaxVersions() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID);
+  }
+
+  /** Returns true if field maxVersions is set (has been assigned a value) and false otherwise */
+  public boolean isSetMaxVersions() {
+    return EncodingUtils.testBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID);
+  }
+
+  public void setMaxVersionsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID, value);
+  }
+
+  public int getMinVersions() {
+    return this.minVersions;
+  }
+
+  public TColumnFamilyDescriptor setMinVersions(int minVersions) {
+    this.minVersions = minVersions;
+    setMinVersionsIsSet(true);
+    return this;
+  }
+
+  public void unsetMinVersions() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MINVERSIONS_ISSET_ID);
+  }
+
+  /** Returns true if field minVersions is set (has been assigned a value) and false otherwise */
+  public boolean isSetMinVersions() {
+    return EncodingUtils.testBit(__isset_bitfield, __MINVERSIONS_ISSET_ID);
+  }
+
+  public void setMinVersionsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MINVERSIONS_ISSET_ID, value);
+  }
+
+  public int getScope() {
+    return this.scope;
+  }
+
+  public TColumnFamilyDescriptor setScope(int scope) {
+    this.scope = scope;
+    setScopeIsSet(true);
+    return this;
+  }
+
+  public void unsetScope() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SCOPE_ISSET_ID);
+  }
+
+  /** Returns true if field scope is set (has been assigned a value) and false otherwise */
+  public boolean isSetScope() {
+    return EncodingUtils.testBit(__isset_bitfield, __SCOPE_ISSET_ID);
+  }
+
+  public void setScopeIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SCOPE_ISSET_ID, value);
+  }
+
+  public int getTimeToLive() {
+    return this.timeToLive;
+  }
+
+  public TColumnFamilyDescriptor setTimeToLive(int timeToLive) {
+    this.timeToLive = timeToLive;
+    setTimeToLiveIsSet(true);
+    return this;
+  }
+
+  public void unsetTimeToLive() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMETOLIVE_ISSET_ID);
+  }
+
+  /** Returns true if field timeToLive is set (has been assigned a value) and false otherwise */
+  public boolean isSetTimeToLive() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIMETOLIVE_ISSET_ID);
+  }
+
+  public void setTimeToLiveIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMETOLIVE_ISSET_ID, value);
+  }
+
+  public boolean isBlockCacheEnabled() {
+    return this.blockCacheEnabled;
+  }
+
+  public TColumnFamilyDescriptor setBlockCacheEnabled(boolean blockCacheEnabled) {
+    this.blockCacheEnabled = blockCacheEnabled;
+    setBlockCacheEnabledIsSet(true);
+    return this;
+  }
+
+  public void unsetBlockCacheEnabled() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BLOCKCACHEENABLED_ISSET_ID);
+  }
+
+  /** Returns true if field blockCacheEnabled is set (has been assigned a value) and false otherwise */
+  public boolean isSetBlockCacheEnabled() {
+    return EncodingUtils.testBit(__isset_bitfield, __BLOCKCACHEENABLED_ISSET_ID);
+  }
+
+  public void setBlockCacheEnabledIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BLOCKCACHEENABLED_ISSET_ID, value);
+  }
+
+  public boolean isCacheBloomsOnWrite() {
+    return this.cacheBloomsOnWrite;
+  }
+
+  public TColumnFamilyDescriptor setCacheBloomsOnWrite(boolean cacheBloomsOnWrite) {
+    this.cacheBloomsOnWrite = cacheBloomsOnWrite;
+    setCacheBloomsOnWriteIsSet(true);
+    return this;
+  }
+
+  public void unsetCacheBloomsOnWrite() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHEBLOOMSONWRITE_ISSET_ID);
+  }
+
+  /** Returns true if field cacheBloomsOnWrite is set (has been assigned a value) and false otherwise */
+  public boolean isSetCacheBloomsOnWrite() {
+    return EncodingUtils.testBit(__isset_bitfield, __CACHEBLOOMSONWRITE_ISSET_ID);
+  }
+
+  public void setCacheBloomsOnWriteIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHEBLOOMSONWRITE_ISSET_ID, value);
+  }
+
+  public boolean isCacheDataOnWrite() {
+    return this.cacheDataOnWrite;
+  }
+
+  public TColumnFamilyDescriptor setCacheDataOnWrite(boolean cacheDataOnWrite) {
+    this.cacheDataOnWrite = cacheDataOnWrite;
+    setCacheDataOnWriteIsSet(true);
+    return this;
+  }
+
+  public void unsetCacheDataOnWrite() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHEDATAONWRITE_ISSET_ID);
+  }
+
+  /** Returns true if field cacheDataOnWrite is set (has been assigned a value) and false otherwise */
+  public boolean isSetCacheDataOnWrite() {
+    return EncodingUtils.testBit(__isset_bitfield, __CACHEDATAONWRITE_ISSET_ID);
+  }
+
+  public void setCacheDataOnWriteIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHEDATAONWRITE_ISSET_ID, value);
+  }
+
+  public boolean isCacheIndexesOnWrite() {
+    return this.cacheIndexesOnWrite;
+  }
+
+  public TColumnFamilyDescriptor setCacheIndexesOnWrite(boolean cacheIndexesOnWrite) {
+    this.cacheIndexesOnWrite = cacheIndexesOnWrite;
+    setCacheIndexesOnWriteIsSet(true);
+    return this;
+  }
+
+  public void unsetCacheIndexesOnWrite() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHEINDEXESONWRITE_ISSET_ID);
+  }
+
+  /** Returns true if field cacheIndexesOnWrite is set (has been assigned a value) and false otherwise */
+  public boolean isSetCacheIndexesOnWrite() {
+    return EncodingUtils.testBit(__isset_bitfield, __CACHEINDEXESONWRITE_ISSET_ID);
+  }
+
+  public void setCacheIndexesOnWriteIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHEINDEXESONWRITE_ISSET_ID, value);
+  }
+
+  public boolean isCompressTags() {
+    return this.compressTags;
+  }
+
+  public TColumnFamilyDescriptor setCompressTags(boolean compressTags) {
+    this.compressTags = compressTags;
+    setCompressTagsIsSet(true);
+    return this;
+  }
+
+  public void unsetCompressTags() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __COMPRESSTAGS_ISSET_ID);
+  }
+
+  /** Returns true if field compressTags is set (has been assigned a value) and false otherwise */
+  public boolean isSetCompressTags() {
+    return EncodingUtils.testBit(__isset_bitfield, __COMPRESSTAGS_ISSET_ID);
+  }
+
+  public void setCompressTagsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __COMPRESSTAGS_ISSET_ID, value);
+  }
+
+  public boolean isEvictBlocksOnClose() {
+    return this.evictBlocksOnClose;
+  }
+
+  public TColumnFamilyDescriptor setEvictBlocksOnClose(boolean evictBlocksOnClose) {
+    this.evictBlocksOnClose = evictBlocksOnClose;
+    setEvictBlocksOnCloseIsSet(true);
+    return this;
+  }
+
+  public void unsetEvictBlocksOnClose() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EVICTBLOCKSONCLOSE_ISSET_ID);
+  }
+
+  /** Returns true if field evictBlocksOnClose is set (has been assigned a value) and false otherwise */
+  public boolean isSetEvictBlocksOnClose() {
+    return EncodingUtils.testBit(__isset_bitfield, __EVICTBLOCKSONCLOSE_ISSET_ID);
+  }
+
+  public void setEvictBlocksOnCloseIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EVICTBLOCKSONCLOSE_ISSET_ID, value);
+  }
+
+  public boolean isInMemory() {
+    return this.inMemory;
+  }
+
+  public TColumnFamilyDescriptor setInMemory(boolean inMemory) {
+    this.inMemory = inMemory;
+    setInMemoryIsSet(true);
+    return this;
+  }
+
+  public void unsetInMemory() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __INMEMORY_ISSET_ID);
+  }
+
+  /** Returns true if field inMemory is set (has been assigned a value) and false otherwise */
+  public boolean isSetInMemory() {
+    return EncodingUtils.testBit(__isset_bitfield, __INMEMORY_ISSET_ID);
+  }
+
+  public void setInMemoryIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __INMEMORY_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((ByteBuffer)value);
+      }
+      break;
+
+    case ATTRIBUTES:
+      if (value == null) {
+        unsetAttributes();
+      } else {
+        setAttributes((Map<ByteBuffer,ByteBuffer>)value);
+      }
+      break;
+
+    case CONFIGURATION:
+      if (value == null) {
+        unsetConfiguration();
+      } else {
+        setConfiguration((Map<String,String>)value);
+      }
+      break;
+
+    case BLOCK_SIZE:
+      if (value == null) {
+        unsetBlockSize();
+      } else {
+        setBlockSize((Integer)value);
+      }
+      break;
+
+    case BLOOMN_FILTER_TYPE:
+      if (value == null) {
+        unsetBloomnFilterType();
+      } else {
+        setBloomnFilterType((TBloomFilterType)value);
+      }
+      break;
+
+    case COMPRESSION_TYPE:
+      if (value == null) {
+        unsetCompressionType();
+      } else {
+        setCompressionType((TCompressionAlgorithm)value);
+      }
+      break;
+
+    case DFS_REPLICATION:
+      if (value == null) {
+        unsetDfsReplication();
+      } else {
+        setDfsReplication((Short)value);
+      }
+      break;
+
+    case DATA_BLOCK_ENCODING:
+      if (value == null) {
+        unsetDataBlockEncoding();
+      } else {
+        setDataBlockEncoding((TDataBlockEncoding)value);
+      }
+      break;
+
+    case KEEP_DELETED_CELLS:
+      if (value == null) {
+        unsetKeepDeletedCells();
+      } else {
+        setKeepDeletedCells((TKeepDeletedCells)value);
+      }
+      break;
+
+    case MAX_VERSIONS:
+      if (value == null) {
+        unsetMaxVersions();
+      } else {
+        setMaxVersions((Integer)value);
+      }
+      break;
+
+    case MIN_VERSIONS:
+      if (value == null) {
+        unsetMinVersions();
+      } else {
+        setMinVersions((Integer)value);
+      }
+      break;
+
+    case SCOPE:
+      if (value == null) {
+        unsetScope();
+      } else {
+        setScope((Integer)value);
+      }
+      break;
+
+    case TIME_TO_LIVE:
+      if (value == null) {
+        unsetTimeToLive();
+      } else {
+        setTimeToLive((Integer)value);
+      }
+      break;
+
+    case BLOCK_CACHE_ENABLED:
+      if (value == null) {
+        unsetBlockCacheEnabled();
+      } else {
+        setBlockCacheEnabled((Boolean)value);
+      }
+      break;
+
+    case CACHE_BLOOMS_ON_WRITE:
+      if (value == null) {
+        unsetCacheBloomsOnWrite();
+      } else {
+        setCacheBloomsOnWrite((Boolean)value);
+      }
+      break;
+
+    case CACHE_DATA_ON_WRITE:
+      if (value == null) {
+        unsetCacheDataOnWrite();
+      } else {
+        setCacheDataOnWrite((Boolean)value);
+      }
+      break;
+
+    case CACHE_INDEXES_ON_WRITE:
+      if (value == null) {
+        unsetCacheIndexesOnWrite();
+      } else {
+        setCacheIndexesOnWrite((Boolean)value);
+      }
+      break;
+
+    case COMPRESS_TAGS:
+      if (value == null) {
+        unsetCompressTags();
+      } else {
+        setCompressTags((Boolean)value);
+      }
+      break;
+
+    case EVICT_BLOCKS_ON_CLOSE:
+      if (value == null) {
+        unsetEvictBlocksOnClose();
+      } else {
+        setEvictBlocksOnClose((Boolean)value);
+      }
+      break;
+
+    case IN_MEMORY:
+      if (value == null) {
+        unsetInMemory();
+      } else {
+        setInMemory((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    case ATTRIBUTES:
+      return getAttributes();
+
+    case CONFIGURATION:
+      return getConfiguration();
+
+    case BLOCK_SIZE:
+      return getBlockSize();
+
+    case BLOOMN_FILTER_TYPE:
+      return getBloomnFilterType();
+
+    case COMPRESSION_TYPE:
+      return getCompressionType();
+
+    case DFS_REPLICATION:
+      return getDfsReplication();
+
+    case DATA_BLOCK_ENCODING:
+      return getDataBlockEncoding();
+
+    case KEEP_DELETED_CELLS:
+      return getKeepDeletedCells();
+
+    case MAX_VERSIONS:
+      return getMaxVersions();
+
+    case MIN_VERSIONS:
+      return getMinVersions();
+
+    case SCOPE:
+      return getScope();
+
+    case TIME_TO_LIVE:
+      return getTimeToLive();
+
+    case BLOCK_CACHE_ENABLED:
+      return isBlockCacheEnabled();
+
+    case CACHE_BLOOMS_ON_WRITE:
+      return isCacheBloomsOnWrite();
+
+    case CACHE_DATA_ON_WRITE:
+      return isCacheDataOnWrite();
+
+    case CACHE_INDEXES_ON_WRITE:
+      return isCacheIndexesOnWrite();
+
+    case COMPRESS_TAGS:
+      return isCompressTags();
+
+    case EVICT_BLOCKS_ON_CLOSE:
+      return isEvictBlocksOnClose();
+
+    case IN_MEMORY:
+      return isInMemory();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    case ATTRIBUTES:
+      return isSetAttributes();
+    case CONFIGURATION:
+      return isSetConfiguration();
+    case BLOCK_SIZE:
+      return isSetBlockSize();
+    case BLOOMN_FILTER_TYPE:
+      return isSetBloomnFilterType();
+    case COMPRESSION_TYPE:
+      return isSetCompressionType();
+    case DFS_REPLICATION:
+      return isSetDfsReplication();
+    case DATA_BLOCK_ENCODING:
+      return isSetDataBlockEncoding();
+    case KEEP_DELETED_CELLS:
+      return isSetKeepDeletedCells();
+    case MAX_VERSIONS:
+      return isSetMaxVersions();
+    case MIN_VERSIONS:
+      return isSetMinVersions();
+    case SCOPE:
+      return isSetScope();
+    case TIME_TO_LIVE:
+      return isSetTimeToLive();
+    case BLOCK_CACHE_ENABLED:
+      return isSetBlockCacheEnabled();
+    case CACHE_BLOOMS_ON_WRITE:
+      return isSetCacheBloomsOnWrite();
+    case CACHE_DATA_ON_WRITE:
+      return isSetCacheDataOnWrite();
+    case CACHE_INDEXES_ON_WRITE:
+      return isSetCacheIndexesOnWrite();
+    case COMPRESS_TAGS:
+      return isSetCompressTags();
+    case EVICT_BLOCKS_ON_CLOSE:
+      return isSetEvictBlocksOnClose();
+    case IN_MEMORY:
+      return isSetInMemory();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TColumnFamilyDescriptor)
+      return this.equals((TColumnFamilyDescriptor)that);
+    return false;
+  }
+
+  public boolean equals(TColumnFamilyDescriptor that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_attributes = true && this.isSetAttributes();
+    boolean that_present_attributes = true && that.isSetAttributes();
+    if (this_present_attributes || that_present_attributes) {
+      if (!(this_present_attributes && that_present_attributes))
+        return false;
+      if (!this.attributes.equals(that.attributes))
+        return false;
+    }
+
+    boolean this_present_configuration = true && this.isSetConfiguration();
+    boolean that_present_configuration = true && that.isSetConfiguration();
+    if (this_present_configuration || that_present_configuration) {
+      if (!(this_present_configuration && that_present_configuration))
+        return false;
+      if (!this.configuration.equals(that.configuration))
+        return false;
+    }
+
+    boolean this_present_blockSize = true && this.isSetBlockSize();
+    boolean that_present_blockSize = true && that.isSetBlockSize();
+    if (this_present_blockSize || that_present_blockSize) {
+      if (!(this_present_blockSize && that_present_blockSize))
+        return false;
+      if (this.blockSize != that.blockSize)
+        return false;
+    }
+
+    boolean this_present_bloomnFilterType = true && this.isSetBloomnFilterType();
+    boolean that_present_bloomnFilterType = true && that.isSetBloomnFilterType();
+    if (this_present_bloomnFilterType || that_present_bloomnFilterType) {
+      if (!(this_present_bloomnFilterType && that_present_bloomnFilterType))
+        return false;
+      if (!this.bloomnFilterType.equals(that.bloomnFilterType))
+        return false;
+    }
+
+    boolean this_present_compressionType = true && this.isSetCompressionType();
+    boolean that_present_compressionType = true && that.isSetCompressionType();
+    if (this_present_compressionType || that_present_compressionType) {
+      if (!(this_present_compressionType && that_present_compressionType))
+        return false;
+      if (!this.compressionType.equals(that.compressionType))
+        return false;
+    }
+
+    boolean this_present_dfsReplication = true && this.isSetDfsReplication();
+    boolean that_present_dfsReplication = true && that.isSetDfsReplication();
+    if (this_present_dfsReplication || that_present_dfsReplication) {
+      if (!(this_present_dfsReplication && that_present_dfsReplication))
+        return false;
+      if (this.dfsReplication != that.dfsReplication)
+        return false;
+    }
+
+    boolean this_present_dataBlockEncoding = true && this.isSetDataBlockEncoding();
+    boolean that_present_dataBlockEncoding = true && that.isSetDataBlockEncoding();
+    if (this_present_dataBlockEncoding || that_present_dataBlockEncoding) {
+      if (!(this_present_dataBlockEncoding && that_present_dataBlockEncoding))
+        return false;
+      if (!this.dataBlockEncoding.equals(that.dataBlockEncoding))
+        return false;
+    }
+
+    boolean this_present_keepDeletedCells = true && this.isSetKeepDeletedCells();
+    boolean that_present_keepDeletedCells = true && that.isSetKeepDeletedCells();
+    if (this_present_keepDeletedCells || that_present_keepDeletedCells) {
+      if (!(this_present_keepDeletedCells && that_present_keepDeletedCells))
+        return false;
+      if (!this.keepDeletedCells.equals(that.keepDeletedCells))
+        return false;
+    }
+
+    boolean this_present_maxVersions = true && this.isSetMaxVersions();
+    boolean that_present_maxVersions = true && that.isSetMaxVersions();
+    if (this_present_maxVersions || that_present_maxVersions) {
+      if (!(this_present_maxVersions && that_present_maxVersions))
+        return false;
+      if (this.maxVersions != that.maxVersions)
+        return false;
+    }
+
+    boolean this_present_minVersions = true && this.isSetMinVersions();
+    boolean that_present_minVersions = true && that.isSetMinVersions();
+    if (this_present_minVersions || that_present_minVersions) {
+      if (!(this_present_minVersions && that_present_minVersions))
+        return false;
+      if (this.minVersions != that.minVersions)
+        return false;
+    }
+
+    boolean this_present_scope = true && this.isSetScope();
+    boolean that_present_scope = true && that.isSetScope();
+    if (this_present_scope || that_present_scope) {
+      if (!(this_present_scope && that_present_scope))
+        return false;
+      if (this.scope != that.scope)
+        return false;
+    }
+
+    boolean this_present_timeToLive = true && this.isSetTimeToLive();
+    boolean that_present_timeToLive = true && that.isSetTimeToLive();
+    if (this_present_timeToLive || that_present_timeToLive) {
+      if (!(this_present_timeToLive && that_present_timeToLive))
+        return false;
+      if (this.timeToLive != that.timeToLive)
+        return false;
+    }
+
+    boolean this_present_blockCacheEnabled = true && this.isSetBlockCacheEnabled();
+    boolean that_present_blockCacheEnabled = true && that.isSetBlockCacheEnabled();
+    if (this_present_blockCacheEnabled || that_present_blockCacheEnabled) {
+      if (!(this_present_blockCacheEnabled && that_present_blockCacheEnabled))
+        return false;
+      if (this.blockCacheEnabled != that.blockCacheEnabled)
+        return false;
+    }
+
+    boolean this_present_cacheBloomsOnWrite = true && this.isSetCacheBloomsOnWrite();
+    boolean that_present_cacheBloomsOnWrite = true && that.isSetCacheBloomsOnWrite();
+    if (this_present_cacheBloomsOnWrite || that_present_cacheBloomsOnWrite) {
+      if (!(this_present_cacheBloomsOnWrite && that_present_cacheBloomsOnWrite))
+        return false;
+      if (this.cacheBloomsOnWrite != that.cacheBloomsOnWrite)
+        return false;
+    }
+
+    boolean this_present_cacheDataOnWrite = true && this.isSetCacheDataOnWrite();
+    boolean that_present_cacheDataOnWrite = true && that.isSetCacheDataOnWrite();
+    if (this_present_cacheDataOnWrite || that_present_cacheDataOnWrite) {
+      if (!(this_present_cacheDataOnWrite && that_present_cacheDataOnWrite))
+        return false;
+      if (this.cacheDataOnWrite != that.cacheDataOnWrite)
+        return false;
+    }
+
+    boolean this_present_cacheIndexesOnWrite = true && this.isSetCacheIndexesOnWrite();
+    boolean that_present_cacheIndexesOnWrite = true && that.isSetCacheIndexesOnWrite();
+    if (this_present_cacheIndexesOnWrite || that_present_cacheIndexesOnWrite) {
+      if (!(this_present_cacheIndexesOnWrite && that_present_cacheIndexesOnWrite))
+        return false;
+      if (this.cacheIndexesOnWrite != that.cacheIndexesOnWrite)
+        return false;
+    }
+
+    boolean this_present_compressTags = true && this.isSetCompressTags();
+    boolean that_present_compressTags = true && that.isSetCompressTags();
+    if (this_present_compressTags || that_present_compressTags) {
+      if (!(this_present_compressTags && that_present_compressTags))
+        return false;
+      if (this.compressTags != that.compressTags)
+        return false;
+    }
+
+    boolean this_present_evictBlocksOnClose = true && this.isSetEvictBlocksOnClose();
+    boolean that_present_evictBlocksOnClose = true && that.isSetEvictBlocksOnClose();
+    if (this_present_evictBlocksOnClose || that_present_evictBlocksOnClose) {
+      if (!(this_present_evictBlocksOnClose && that_present_evictBlocksOnClose))
+        return false;
+      if (this.evictBlocksOnClose != that.evictBlocksOnClose)
+        return false;
+    }
+
+    boolean this_present_inMemory = true && this.isSetInMemory();
+    boolean that_present_inMemory = true && that.isSetInMemory();
+    if (this_present_inMemory || that_present_inMemory) {
+      if (!(this_present_inMemory && that_present_inMemory))
+        return false;
+      if (this.inMemory != that.inMemory)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_attributes = true && (isSetAttributes());
+    list.add(present_attributes);
+    if (present_attributes)
+      list.add(attributes);
+
+    boolean present_configuration = true && (isSetConfiguration());
+    list.add(present_configuration);
+    if (present_configuration)
+      list.add(configuration);
+
+    boolean present_blockSize = true && (isSetBlockSize());
+    list.add(present_blockSize);
+    if (present_blockSize)
+      list.add(blockSize);
+
+    boolean present_bloomnFilterType = true && (isSetBloomnFilterType());
+    list.add(present_bloomnFilterType);
+    if (present_bloomnFilterType)
+      list.add(bloomnFilterType.getValue());
+
+    boolean present_compressionType = true && (isSetCompressionType());
+    list.add(present_compressionType);
+    if (present_compressionType)
+      list.add(compressionType.getValue());
+
+    boolean present_dfsReplication = true && (isSetDfsReplication());
+    list.add(present_dfsReplication);
+    if (present_dfsReplication)
+      list.add(dfsReplication);
+
+    boolean present_dataBlockEncoding = true && (isSetDataBlockEncoding());
+    list.add(present_dataBlockEncoding);
+    if (present_dataBlockEncoding)
+      list.add(dataBlockEncoding.getValue());
+
+    boolean present_keepDeletedCells = true && (isSetKeepDeletedCells());
+    list.add(present_keepDeletedCells);
+    if (present_keepDeletedCells)
+      list.add(keepDeletedCells.getValue());
+
+    boolean present_maxVersions = true && (isSetMaxVersions());
+    list.add(present_maxVersions);
+    if (present_maxVersions)
+      list.add(maxVersions);
+
+    boolean present_minVersions = true && (isSetMinVersions());
+    list.add(present_minVersions);
+    if (present_minVersions)
+      list.add(minVersions);
+
+    boolean present_scope = true && (isSetScope());
+    list.add(present_scope);
+    if (present_scope)
+      list.add(scope);
+
+    boolean present_timeToLive = true && (isSetTimeToLive());
+    list.add(present_timeToLive);
+    if (present_timeToLive)
+      list.add(timeToLive);
+
+    boolean present_blockCacheEnabled = true && (isSetBlockCacheEnabled());
+    list.add(present_blockCacheEnabled);
+    if (present_blockCacheEnabled)
+      list.add(blockCacheEnabled);
+
+    boolean present_cacheBloomsOnWrite = true && (isSetCacheBloomsOnWrite());
+    list.add(present_cacheBloomsOnWrite);
+    if (present_cacheBloomsOnWrite)
+      list.add(cacheBloomsOnWrite);
+
+    boolean present_cacheDataOnWrite = true && (isSetCacheDataOnWrite());
+    list.add(present_cacheDataOnWrite);
+    if (present_cacheDataOnWrite)
+      list.add(cacheDataOnWrite);
+
+    boolean present_cacheIndexesOnWrite = true && (isSetCacheIndexesOnWrite());
+    list.add(present_cacheIndexesOnWrite);
+    if (present_cacheIndexesOnWrite)
+      list.add(cacheIndexesOnWrite);
+
+    boolean present_compressTags = true && (isSetCompressTags());
+    list.add(present_compressTags);
+    if (present_compressTags)
+      list.add(compressTags);
+
+    boolean present_evictBlocksOnClose = true && (isSetEvictBlocksOnClose());
+    list.add(present_evictBlocksOnClose);
+    if (present_evictBlocksOnClose)
+      list.add(evictBlocksOnClose);
+
+    boolean present_inMemory = true && (isSetInMemory());
+    list.add(present_inMemory);
+    if (present_inMemory)
+      list.add(inMemory);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TColumnFamilyDescriptor other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAttributes()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(other.isSetConfiguration());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetConfiguration()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, other.configuration);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetBlockSize()).compareTo(other.isSetBlockSize());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetBlockSize()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.blockSize, other.blockSize);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetBloomnFilterType()).compareTo(other.isSetBloomnFilterType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetBloomnFilterType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bloomnFilterType, other.bloomnFilterType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCompressionType()).compareTo(other.isSetCompressionType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCompressionType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compressionType, other.compressionType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDfsReplication()).compareTo(other.isSetDfsReplication());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDfsReplication()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dfsReplication, other.dfsReplication);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDataBlockEncoding()).compareTo(other.isSetDataBlockEncoding());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDataBlockEncoding()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dataBlockEncoding, other.dataBlockEncoding);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetKeepDeletedCells()).compareTo(other.isSetKeepDeletedCells());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetKeepDeletedCells()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keepDeletedCells, other.keepDeletedCells);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetMaxVersions()).compareTo(other.isSetMaxVersions());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMaxVersions()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxVersions, other.maxVersions);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetMinVersions()).compareTo(other.isSetMinVersions());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMinVersions()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.minVersions, other.minVersions);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetScope()).compareTo(other.isSetScope());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetScope()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scope, other.scope);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTimeToLive()).compareTo(other.isSetTimeToLive());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTimeToLive()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timeToLive, other.timeToLive);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetBlockCacheEnabled()).compareTo(other.isSetBlockCacheEnabled());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetBlockCacheEnabled()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.blockCacheEnabled, other.blockCacheEnabled);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCacheBloomsOnWrite()).compareTo(other.isSetCacheBloomsOnWrite());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCacheBloomsOnWrite()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cacheBloomsOnWrite, other.cacheBloomsOnWrite);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCacheDataOnWrite()).compareTo(other.isSetCacheDataOnWrite());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCacheDataOnWrite()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cacheDataOnWrite, other.cacheDataOnWrite);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCacheIndexesOnWrite()).compareTo(other.isSetCacheIndexesOnWrite());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCacheIndexesOnWrite()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cacheIndexesOnWrite, other.cacheIndexesOnWrite);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCompressTags()).compareTo(other.isSetCompressTags());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCompressTags()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compressTags, other.compressTags);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetEvictBlocksOnClose()).compareTo(other.isSetEvictBlocksOnClose());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetEvictBlocksOnClose()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.evictBlocksOnClose, other.evictBlocksOnClose);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetInMemory()).compareTo(other.isSetInMemory());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetInMemory()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.inMemory, other.inMemory);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TColumnFamilyDescriptor(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.name, sb);
+    }
+    first = false;
+    if (isSetAttributes()) {
+      if (!first) sb.append(", ");
+      sb.append("attributes:");
+      if (this.attributes == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.attributes);
+      }
+      first = false;
+    }
+    if (isSetConfiguration()) {
+      if (!first) sb.append(", ");
+      sb.append("configuration:");
+      if (this.configuration == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.configuration);
+      }
+      first = false;
+    }
+    if (isSetBlockSize()) {
+      if (!first) sb.append(", ");
+      sb.append("blockSize:");
+      sb.append(this.blockSize);
+      first = false;
+    }
+    if (isSetBloomnFilterType()) {
+      if (!first) sb.append(", ");
+      sb.append("bloomnFilterType:");
+      if (this.bloomnFilterType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.bloomnFilterType);
+      }
+      first = false;
+    }
+    if (isSetCompressionType()) {
+      if (!first) sb.append(", ");
+      sb.append("compressionType:");
+      if (this.compressionType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.compressionType);
+      }
+      first = false;
+    }
+    if (isSetDfsReplication()) {
+      if (!first) sb.append(", ");
+      sb.append("dfsReplication:");
+      sb.append(this.dfsReplication);
+      first = false;
+    }
+    if (isSetDataBlockEncoding()) {
+      if (!first) sb.append(", ");
+      sb.append("dataBlockEncoding:");
+      if (this.dataBlockEncoding == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.dataBlockEncoding);
+      }
+      first = false;
+    }
+    if (isSetKeepDeletedCells()) {
+      if (!first) sb.append(", ");
+      sb.append("keepDeletedCells:");
+      if (this.keepDeletedCells == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.keepDeletedCells);
+      }
+      first = false;
+    }
+    if (isSetMaxVersions()) {
+      if (!first) sb.append(", ");
+      sb.append("maxVersions:");
+      sb.append(this.maxVersions);
+      first = false;
+    }
+    if (isSetMinVersions()) {
+      if (!first) sb.append(", ");
+      sb.append("minVersions:");
+      sb.append(this.minVersions);
+      first = false;
+    }
+    if (isSetScope()) {
+      if (!first) sb.append(", ");
+      sb.append("scope:");
+      sb.append(this.scope);
+      first = false;
+    }
+    if (isSetTimeToLive()) {
+      if (!first) sb.append(", ");
+      sb.append("timeToLive:");
+      sb.append(this.timeToLive);
+      first = false;
+    }
+    if (isSetBlockCacheEnabled()) {
+      if (!first) sb.append(", ");
+      sb.append("blockCacheEnabled:");
+      sb.append(this.blockCacheEnabled);
+      first = false;
+    }
+    if (isSetCacheBloomsOnWrite()) {
+      if (!first) sb.append(", ");
+      sb.append("cacheBloomsOnWrite:");
+      sb.append(this.cacheBloomsOnWrite);
+      first = false;
+    }
+    if (isSetCacheDataOnWrite()) {
+      if (!first) sb.append(", ");
+      sb.append("cacheDataOnWrite:");
+      sb.append(this.cacheDataOnWrite);
+      first = false;
+    }
+    if (isSetCacheIndexesOnWrite()) {
+      if (!first) sb.append(", ");
+      sb.append("cacheIndexesOnWrite:");
+      sb.append(this.cacheIndexesOnWrite);
+      first = false;
+    }
+    if (isSetCompressTags()) {
+      if (!first) sb.append(", ");
+      sb.append("compressTags:");
+      sb.append(this.compressTags);
+      first = false;
+    }
+    if (isSetEvictBlocksOnClose()) {
+      if (!first) sb.append(", ");
+      sb.append("evictBlocksOnClose:");
+      sb.append(this.evictBlocksOnClose);
+      first = false;
+    }
+    if (isSetInMemory()) {
+      if (!first) sb.append(", ");
+      sb.append("inMemory:");
+      sb.append(this.inMemory);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (name == null) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString());
+    }
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TColumnFamilyDescriptorStandardSchemeFactory implements SchemeFactory {
+    public TColumnFamilyDescriptorStandardScheme getScheme() {
+      return new TColumnFamilyDescriptorStandardScheme();
+    }
+  }
+
+  private static class TColumnFamilyDescriptorStandardScheme extends StandardScheme<TColumnFamilyDescriptor> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnFamilyDescriptor struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readBinary();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // ATTRIBUTES
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map142 = iprot.readMapBegin();
+                struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map142.size);
+                ByteBuffer _key143;
+                ByteBuffer _val144;
+                for (int _i145 = 0; _i145 < _map142.size; ++_i145)
+                {
+                  _key143 = iprot.readBinary();
+                  _val144 = iprot.readBinary();
+                  struct.attributes.put(_key143, _val144);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setAttributesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // CONFIGURATION
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map146 = iprot.readMapBegin();
+                struct.configuration = new HashMap<String,String>(2*_map146.size);
+                String _key147;
+                String _val148;
+                for (int _i149 = 0; _i149 < _map146.size; ++_i149)
+                {
+                  _key147 = iprot.readString();
+                  _val148 = iprot.readString();
+                  struct.configuration.put(_key147, _val148);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setConfigurationIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // BLOCK_SIZE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.blockSize = iprot.readI32();
+              struct.setBlockSizeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // BLOOMN_FILTER_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.bloomnFilterType = org.apache.hadoop.hbase.thrift2.generated.TBloomFilterType.findByValue(iprot.readI32());
+              struct.setBloomnFilterTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // COMPRESSION_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.compressionType = org.apache.hadoop.hbase.thrift2.generated.TCompressionAlgorithm.findByValue(iprot.readI32());
+              struct.setCompressionTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // DFS_REPLICATION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I16) {
+              struct.dfsReplication = iprot.readI16();
+              struct.setDfsReplicationIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // DATA_BLOCK_ENCODING
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.dataBlockEncoding = org.apache.hadoop.hbase.thrift2.generated.TDataBlockEncoding.findByValue(iprot.readI32());
+              struct.setDataBlockEncodingIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // KEEP_DELETED_CELLS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.keepDeletedCells = org.apache.hadoop.hbase.thrift2.generated.TKeepDeletedCells.findByValue(iprot.readI32());
+              struct.setKeepDeletedCellsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // MAX_VERSIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.maxVersions = iprot.readI32();
+              struct.setMaxVersionsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 11: // MIN_VERSIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.minVersions = iprot.readI32();
+              struct.setMinVersionsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // SCOPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.scope = iprot.readI32();
+              struct.setScopeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // TIME_TO_LIVE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.timeToLive = iprot.readI32();
+              struct.setTimeToLiveIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 14: // BLOCK_CACHE_ENABLED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.blockCacheEnabled = iprot.readBool();
+              struct.setBlockCacheEnabledIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 15: // CACHE_BLOOMS_ON_WRITE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.cacheBloomsOnWrite = iprot.readBool();
+              struct.setCacheBloomsOnWriteIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 16: // CACHE_DATA_ON_WRITE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.cacheDataOnWrite = iprot.readBool();
+              struct.setCacheDataOnWriteIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 17: // CACHE_INDEXES_ON_WRITE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.cacheIndexesOnWrite = iprot.readBool();
+              struct.setCacheIndexesOnWriteIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 18: // COMPRESS_TAGS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.compressTags = iprot.readBool();
+              struct.setCompressTagsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 19: // EVICT_BLOCKS_ON_CLOSE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.evictBlocksOnClose = iprot.readBool();
+              struct.setEvictBlocksOnCloseIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 20: // IN_MEMORY
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.inMemory = iprot.readBool();
+              struct.setInMemoryIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TColumnFamilyDescriptor struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeBinary(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.attributes != null) {
+        if (struct.isSetAttributes()) {
+          oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size()));
+            for (Map.Entry<ByteBuffer, ByteBuffer> _iter150 : struct.attributes.entrySet())
+            {
+              oprot.writeBinary(_iter150.getKey());
+              oprot.writeBinary(_iter150.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.configuration != null) {
+        if (struct.isSetConfiguration()) {
+          oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size()));
+            for (Map.Entry<String, String> _iter151 : struct.configuration.entrySet())
+            {
+              oprot.writeString(_iter151.getKey());
+              oprot.writeString(_iter151.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetBlockSize()) {
+        oprot.writeFieldBegin(BLOCK_SIZE_FIELD_DESC);
+        oprot.writeI32(struct.blockSize);
+        oprot.writeFieldEnd();
+      }
+      if (struct.bloomnFilterType != null) {
+        if (struct.isSetBloomnFilterType()) {
+          oprot.writeFieldBegin(BLOOMN_FILTER_TYPE_FIELD_DESC);
+          oprot.writeI32(struct.bloomnFilterType.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.compressionType != null) {
+        if (struct.isSetCompressionType()) {
+          oprot.writeFieldBegin(COMPRESSION_TYPE_FIELD_DESC);
+          oprot.writeI32(struct.compressionType.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetDfsReplication()) {
+        oprot.writeFieldBegin(DFS_REPLICATION_FIELD_DESC);
+        oprot.writeI16(struct.dfsReplication);
+        oprot.writeFieldEnd();
+      }
+      if (struct.dataBlockEncoding != null) {
+        if (struct.isSetDataBlockEncoding()) {
+          oprot.writeFieldBegin(DATA_BLOCK_ENCODING_FIELD_DESC);
+          oprot.writeI32(struct.dataBlockEncoding.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.keepDeletedCells != null) {
+        if (struct.isSetKeepDeletedCells()) {
+          oprot.writeFieldBegin(KEEP_DELETED_CELLS_FIELD_DESC);
+          oprot.writeI32(struct.keepDeletedCells.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetMaxVersions()) {
+        oprot.writeFieldBegin(MAX_VERSIONS_FIELD_DESC);
+        oprot.writeI32(struct.maxVersions);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetMinVersions()) {
+        oprot.writeFieldBegin(MIN_VERSIONS_FIELD_DESC);
+        oprot.writeI32(struct.minVersions);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetScope()) {
+        oprot.writeFieldBegin(SCOPE_FIELD_DESC);
+        oprot.writeI32(struct.scope);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetTimeToLive()) {
+        oprot.writeFieldBegin(TIME_TO_LIVE_FIELD_DESC);
+        oprot.writeI32(struct.timeToLive);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetBlockCacheEnabled()) {
+        oprot.writeFieldBegin(BLOCK_CACHE_ENABLED_FIELD_DESC);
+        oprot.writeBool(struct.blockCacheEnabled);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetCacheBloomsOnWrite()) {
+        oprot.writeFieldBegin(CACHE_BLOOMS_ON_WRITE_FIELD_DESC);
+        oprot.writeBool(struct.cacheBloomsOnWrite);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetCacheDataOnWrite()) {
+        oprot.writeFieldBegin(CACHE_DATA_ON_WRITE_FIELD_DESC);
+        oprot.writeBool(struct.cacheDataOnWrite);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetCacheIndexesOnWrite()) {
+        oprot.writeFieldBegin(CACHE_INDEXES_ON_WRITE_FIELD_DESC);
+        oprot.writeBool(struct.cacheIndexesOnWrite);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetCompressTags()) {
+        oprot.writeFieldBegin(COMPRESS_TAGS_FIELD_DESC);
+        oprot.writeBool(struct.compressTags);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetEvictBlocksOnClose()) {
+        oprot.writeFieldBegin(EVICT_BLOCKS_ON_CLOSE_FIELD_DESC);
+        oprot.writeBool(struct.evictBlocksOnClose);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetInMemory()) {
+        oprot.writeFieldBegin(IN_MEMORY_FIELD_DESC);
+        oprot.writeBool(struct.inMemory);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TColumnFamilyDescriptorTupleSchemeFactory implements SchemeFactory {
+    public TColumnFamilyDescriptorTupleScheme getScheme() {
+      return new TColumnFamilyDescriptorTupleScheme();
+    }
+  }
+
+  private static class TColumnFamilyDescriptorTupleScheme extends TupleScheme<TColumnFamilyDescriptor> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TColumnFamilyDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeBinary(struct.name);
+      BitSet optionals = new BitSet();
+      if (struct.isSetAttributes()) {
+        optionals.set(0);
+      }
+      if (struct.isSetConfiguration()) {
+        optionals.set(1);
+      }
+      if (struct.isSetBlockSize()) {
+        optionals.set(2);
+      }
+      if (struct.isSetBloomnFilterType()) {
+        optionals.set(3);
+      }
+      if (struct.isSetCompressionType()) {
+        optionals.set(4);
+      }
+      if (struct.isSetDfsReplication()) {
+        optionals.set(5);
+      }
+      if (struct.isSetDataBlockEncoding()) {
+        optionals.set(6);
+      }
+      if (struct.isSetKeepDeletedCells()) {
+        optionals.set(7);
+      }
+      if (struct.isSetMaxVersions()) {
+        optionals.set(8);
+      }
+      if (struct.isSetMinVersions()) {
+        optionals.set(9);
+      }
+      if (struct.isSetScope()) {
+        optionals.set(10);
+      }
+      if (struct.isSetTimeToLive()) {
+        optionals.set(11);
+      }
+      if (struct.isSetBlockCacheEnabled()) {
+        optionals.set(12);
+      }
+      if (struct.isSetCacheBloomsOnWrite()) {
+        optionals.set(13);
+      }
+      if (struct.isSetCacheDataOnWrite()) {
+        optionals.set(14);
+      }
+      if (struct.isSetCacheIndexesOnWrite()) {
+        optionals.set(15);
+      }
+      if (struct.isSetCompressTags()) {
+        optionals.set(16);
+      }
+      if (struct.isSetEvictBlocksOnClose()) {
+        optionals.set(17);
+      }
+      if (struct.isSetInMemory()) {
+        optionals.set(18);
+      }
+      oprot.writeBitSet(optionals, 19);
+      if (struct.isSetAttributes()) {
+        {
+          oprot.writeI32(struct.attributes.size());
+          for (Map.Entry<ByteBuffer, ByteBuffer> _iter152 : struct.attributes.entrySet())
+          {
+            oprot.writeBinary(_iter152.getKey());
+            oprot.writeBinary(_iter152.getValue());
+          }
+        }
+      }
+      if (struct.isSetConfiguration()) {
+        {
+          oprot.writeI32(struct.configuration.size());
+          for (Map.Entry<String, String> _iter153 : struct.configuration.entrySet())
+          {
+            oprot.writeString(_iter153.getKey());
+            oprot.writeString(_iter153.getValue());
+          }
+        }
+      }
+      if (struct.isSetBlockSize()) {
+        oprot.writeI32(struct.blockSize);
+      }
+      if (struct.isSetBloomnFilterType()) {
+        oprot.writeI32(struct.bloomnFilterType.getValue());
+      }
+      if (struct.isSetCompressionType()) {
+        oprot.writeI32(struct.compressionType.getValue());
+      }
+      if (struct.isSetDfsReplication()) {
+        oprot.writeI16(struct.dfsReplication);
+      }
+      if (struct.isSetDataBlockEncoding()) {
+        oprot.writeI32(struct.dataBlockEncoding.getValue());
+      }
+      if (struct.isSetKeepDeletedCells()) {
+        oprot.writeI32(struct.keepDeletedCells.getValue());
+      }
+      if (struct.isSetMaxVersions()) {
+        oprot.writeI32(struct.maxVersions);
+      }
+      if (struct.isSetMinVersions()) {
+        oprot.writeI32(struct.minVersions);
+      }
+      if (struct.isSetScope()) {
+        oprot.writeI3

<TRUNCATED>

[10/47] hbase git commit: Update downloads.xml for release 1.4.9

Posted by zh...@apache.org.
Update downloads.xml for release 1.4.9


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

Branch: refs/heads/HBASE-21512
Commit: 2b003c5d685160eeaf90387e887b433dadb8695e
Parents: 1b08ba7
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Dec 14 13:54:57 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Dec 14 13:54:57 2018 -0800

----------------------------------------------------------------------
 src/site/xdoc/downloads.xml | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2b003c5d/src/site/xdoc/downloads.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index 5d3f2a6..4bb7f51 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -88,23 +88,23 @@ under the License.
     </tr>
     <tr>
       <td style="test-align: left">
-        1.4.8
+        1.4.9
       </td>
       <td style="test-align: left">
-        2018/10/08
+        2018/12/14
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/1.4.8/compat-check-report.html">1.4.7 vs 1.4.8</a>
+        <a href="https://apache.org/dist/hbase/1.4.9/compat-check-report.html">1.4.8 vs 1.4.9</a>
       </td>
       <td style="test-align: left">
-        <a href="https://github.com/apache/hbase/blob/rel/1.4.8/CHANGES.txt">Changes</a>
+        <a href="https://github.com/apache/hbase/blob/rel/1.4.9/CHANGES.txt">Changes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://s.apache.org/hbase-1.4.8-jira-release-notes">Release Notes</a>
+        <a href="https://s.apache.org/hbase-1.4.9-jira-release-notes">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://www.apache.org/dyn/closer.lua/hbase/1.4.8/hbase-1.4.8-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/1.4.8/hbase-1.4.8-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.4.8/hbase-1.4.8-src.tar.gz.asc">asc</a>) <br />
-        <a href="https://www.apache.org/dyn/closer.lua/hbase/1.4.8/hbase-1.4.8-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/1.4.8/hbase-1.4.8-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.4.8/hbase-1.4.8-bin.tar.gz.asc">asc</a>)
+        <a href="https://www.apache.org/dyn/closer.lua/hbase/1.4.9/hbase-1.4.9-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-src.tar.gz.asc">asc</a>) <br />
+        <a href="https://www.apache.org/dyn/closer.lua/hbase/1.4.9/hbase-1.4.9-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-bin.tar.gz.asc">asc</a>)
       </td>
     </tr>
     <tr>


[23/47] hbase git commit: HBASE-21401 Sanity check when constructing the KeyValue

Posted by zh...@apache.org.
HBASE-21401 Sanity check when constructing the KeyValue


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

Branch: refs/heads/HBASE-21512
Commit: 12786f80c14c6f2c3c111a55bbf431fb2e81e828
Parents: ad81938
Author: huzheng <op...@gmail.com>
Authored: Sat Oct 27 16:57:01 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Fri Dec 21 18:01:35 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/KeyValue.java  |  12 +-
 .../org/apache/hadoop/hbase/KeyValueUtil.java   | 148 +++++++++-
 .../hadoop/hbase/codec/KeyValueCodec.java       |   3 +-
 .../hbase/codec/KeyValueCodecWithTags.java      |   2 +-
 .../org/apache/hadoop/hbase/TestKeyValue.java   | 295 +++++++++----------
 .../hadoop/hbase/regionserver/HStore.java       |   1 -
 .../io/encoding/TestDataBlockEncoders.java      |   2 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |   4 +-
 .../hadoop/hbase/io/hfile/TestHFileSeek.java    |   9 +-
 9 files changed, 290 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index f7f6c0d..bdaefff 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -252,6 +252,15 @@ public class KeyValue implements ExtendedCell, Cloneable {
     }
 
     /**
+     * True to indicate that the byte b is a valid type.
+     * @param b byte to check
+     * @return true or false
+     */
+    static boolean isValidType(byte b) {
+      return codeArray[b & 0xff] != null;
+    }
+
+    /**
      * Cannot rely on enum ordinals . They change if item is removed or moved.
      * Do our own codes.
      * @param b
@@ -331,7 +340,8 @@ public class KeyValue implements ExtendedCell, Cloneable {
    * @param offset offset to start of the KeyValue
    * @param length length of the KeyValue
    */
-  public KeyValue(final byte [] bytes, final int offset, final int length) {
+  public KeyValue(final byte[] bytes, final int offset, final int length) {
+    KeyValueUtil.checkKeyValueBytes(bytes, offset, length, true);
     this.bytes = bytes;
     this.offset = offset;
     this.length = length;

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index 1b61d1e..16ebdbf 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -518,17 +518,145 @@ public class KeyValueUtil {
     return (long) length + Bytes.SIZEOF_INT;
   }
 
+  static String bytesToHex(byte[] buf, int offset, int length) {
+    return ", KeyValueBytesHex=" + Bytes.toStringBinary(buf, offset, length) + ", offset=" + offset
+        + ", length=" + length;
+  }
+
+  static void checkKeyValueBytes(byte[] buf, int offset, int length, boolean withTags) {
+    int pos = offset, endOffset = offset + length;
+    // check the key
+    if (pos + Bytes.SIZEOF_INT > endOffset) {
+      throw new IllegalArgumentException(
+          "Overflow when reading key length at position=" + pos + bytesToHex(buf, offset, length));
+    }
+    int keyLen = Bytes.toInt(buf, pos, Bytes.SIZEOF_INT);
+    pos += Bytes.SIZEOF_INT;
+    if (keyLen <= 0 || pos + keyLen > endOffset) {
+      throw new IllegalArgumentException(
+          "Invalid key length in KeyValue. keyLength=" + keyLen + bytesToHex(buf, offset, length));
+    }
+    // check the value
+    if (pos + Bytes.SIZEOF_INT > endOffset) {
+      throw new IllegalArgumentException("Overflow when reading value length at position=" + pos
+          + bytesToHex(buf, offset, length));
+    }
+    int valLen = Bytes.toInt(buf, pos, Bytes.SIZEOF_INT);
+    pos += Bytes.SIZEOF_INT;
+    if (valLen < 0 || pos + valLen > endOffset) {
+      throw new IllegalArgumentException("Invalid value length in KeyValue, valueLength=" + valLen
+          + bytesToHex(buf, offset, length));
+    }
+    // check the row
+    if (pos + Bytes.SIZEOF_SHORT > endOffset) {
+      throw new IllegalArgumentException(
+          "Overflow when reading row length at position=" + pos + bytesToHex(buf, offset, length));
+    }
+    short rowLen = Bytes.toShort(buf, pos, Bytes.SIZEOF_SHORT);
+    pos += Bytes.SIZEOF_SHORT;
+    if (rowLen < 0 || pos + rowLen > endOffset) {
+      throw new IllegalArgumentException(
+          "Invalid row length in KeyValue, rowLength=" + rowLen + bytesToHex(buf, offset, length));
+    }
+    pos += rowLen;
+    // check the family
+    if (pos + Bytes.SIZEOF_BYTE > endOffset) {
+      throw new IllegalArgumentException("Overflow when reading family length at position=" + pos
+          + bytesToHex(buf, offset, length));
+    }
+    int familyLen = buf[pos];
+    pos += Bytes.SIZEOF_BYTE;
+    if (familyLen < 0 || pos + familyLen > endOffset) {
+      throw new IllegalArgumentException("Invalid family length in KeyValue, familyLength="
+          + familyLen + bytesToHex(buf, offset, length));
+    }
+    pos += familyLen;
+    // check the qualifier
+    int qualifierLen = keyLen - Bytes.SIZEOF_SHORT - rowLen - Bytes.SIZEOF_BYTE - familyLen
+        - Bytes.SIZEOF_LONG - Bytes.SIZEOF_BYTE;
+    if (qualifierLen < 0 || pos + qualifierLen > endOffset) {
+      throw new IllegalArgumentException("Invalid qualifier length in KeyValue, qualifierLen="
+          + qualifierLen + bytesToHex(buf, offset, length));
+    }
+    pos += qualifierLen;
+    // check the timestamp
+    if (pos + Bytes.SIZEOF_LONG > endOffset) {
+      throw new IllegalArgumentException(
+          "Overflow when reading timestamp at position=" + pos + bytesToHex(buf, offset, length));
+    }
+    long timestamp = Bytes.toLong(buf, pos, Bytes.SIZEOF_LONG);
+    if (timestamp < 0) {
+      throw new IllegalArgumentException(
+          "Timestamp cannot be negative, ts=" + timestamp + bytesToHex(buf, offset, length));
+    }
+    pos += Bytes.SIZEOF_LONG;
+    // check the type
+    if (pos + Bytes.SIZEOF_BYTE > endOffset) {
+      throw new IllegalArgumentException(
+          "Overflow when reading type at position=" + pos + bytesToHex(buf, offset, length));
+    }
+    byte type = buf[pos];
+    if (!Type.isValidType(type)) {
+      throw new IllegalArgumentException(
+          "Invalid type in KeyValue, type=" + type + bytesToHex(buf, offset, length));
+    }
+    pos += Bytes.SIZEOF_BYTE;
+    // check the value
+    if (pos + valLen > endOffset) {
+      throw new IllegalArgumentException(
+          "Overflow when reading value part at position=" + pos + bytesToHex(buf, offset, length));
+    }
+    pos += valLen;
+    // check the tags
+    if (withTags) {
+      if (pos == endOffset) {
+        // withTags is true but no tag in the cell.
+        return;
+      }
+      if (pos + Bytes.SIZEOF_SHORT > endOffset) {
+        throw new IllegalArgumentException("Overflow when reading tags length at position=" + pos
+            + bytesToHex(buf, offset, length));
+      }
+      short tagsLen = Bytes.toShort(buf, pos);
+      pos += Bytes.SIZEOF_SHORT;
+      if (tagsLen < 0 || pos + tagsLen > endOffset) {
+        throw new IllegalArgumentException("Invalid tags length in KeyValue at position="
+            + (pos - Bytes.SIZEOF_SHORT) + bytesToHex(buf, offset, length));
+      }
+      int tagsEndOffset = pos + tagsLen;
+      for (; pos < tagsEndOffset;) {
+        if (pos + Tag.TAG_LENGTH_SIZE > endOffset) {
+          throw new IllegalArgumentException("Overflow when reading tag length at position=" + pos
+              + bytesToHex(buf, offset, length));
+        }
+        short tagLen = Bytes.toShort(buf, pos);
+        pos += Tag.TAG_LENGTH_SIZE;
+        // tagLen contains one byte tag type, so must be not less than 1.
+        if (tagLen < 1 || pos + tagLen > endOffset) {
+          throw new IllegalArgumentException(
+              "Invalid tag length at position=" + (pos - Tag.TAG_LENGTH_SIZE) + ", tagLength="
+                  + tagLen + bytesToHex(buf, offset, length));
+        }
+        pos += tagLen;
+      }
+    }
+    if (pos != endOffset) {
+      throw new IllegalArgumentException("Some redundant bytes in KeyValue's buffer, startOffset="
+          + pos + ", endOffset=" + endOffset + bytesToHex(buf, offset, length));
+    }
+  }
+
   /**
    * Create a KeyValue reading from the raw InputStream. Named
-   * <code>iscreate</code> so doesn't clash with {@link #create(DataInput)}
-   *
-   * @param in
+   * <code>createKeyValueFromInputStream</code> so doesn't clash with {@link #create(DataInput)}
+   * @param in inputStream to read.
    * @param withTags whether the keyvalue should include tags are not
-   * @return Created KeyValue OR if we find a length of zero, we will return
-   *         null which can be useful marking a stream as done.
+   * @return Created KeyValue OR if we find a length of zero, we will return null which can be
+   *         useful marking a stream as done.
    * @throws IOException
    */
-  public static KeyValue iscreate(final InputStream in, boolean withTags) throws IOException {
+  public static KeyValue createKeyValueFromInputStream(InputStream in, boolean withTags)
+      throws IOException {
     byte[] intBytes = new byte[Bytes.SIZEOF_INT];
     int bytesRead = 0;
     while (bytesRead < intBytes.length) {
@@ -541,14 +669,10 @@ public class KeyValueUtil {
       }
       bytesRead += n;
     }
-    // TODO: perhaps some sanity check is needed here.
     byte[] bytes = new byte[Bytes.toInt(intBytes)];
     IOUtils.readFully(in, bytes, 0, bytes.length);
-    if (withTags) {
-      return new KeyValue(bytes, 0, bytes.length);
-    } else {
-      return new NoTagsKeyValue(bytes, 0, bytes.length);
-    }
+    return withTags ? new KeyValue(bytes, 0, bytes.length)
+        : new NoTagsKeyValue(bytes, 0, bytes.length);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
index 9a5db3c..19a80b0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
@@ -72,7 +72,7 @@ public class KeyValueCodec implements Codec {
     @Override
     protected Cell parseCell() throws IOException {
       // No tags here
-      return KeyValueUtil.iscreate(in, false);
+      return KeyValueUtil.createKeyValueFromInputStream(in, false);
     }
   }
 
@@ -114,7 +114,6 @@ public class KeyValueCodec implements Codec {
       // We know there is not going to be any tags.
       return new NoTagsByteBufferKeyValue(bb, pos, len);
     }
-
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
index bf9b375..34c2b87 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
@@ -78,7 +78,7 @@ public class KeyValueCodecWithTags implements Codec {
     @Override
     protected Cell parseCell() throws IOException {
       // create KeyValue with tags
-      return KeyValueUtil.iscreate(in, true);
+      return KeyValueUtil.createKeyValueFromInputStream(in, true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
index 167a030..2165b38 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
@@ -18,11 +18,13 @@
  */
 package org.apache.hadoop.hbase;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -590,181 +592,152 @@ public class TestKeyValue {
 
   @Test
   public void testKeyValueSerialization() throws Exception {
-    KeyValue kvA1 = new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
-        Bytes.toBytes("1"));
-    KeyValue kvA2 = new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
-        Bytes.toBytes("2"));
-    MockKeyValue mkvA1 = new MockKeyValue(kvA1);
-    MockKeyValue mkvA2 = new MockKeyValue(kvA2);
+    KeyValue[] keyValues = new KeyValue[] {
+        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
+            Bytes.toBytes("1")),
+        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
+            Bytes.toBytes("2")),
+        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
+            System.currentTimeMillis(), Bytes.toBytes("2"),
+            new Tag[] { new ArrayBackedTag((byte) 120, "tagA"),
+                new ArrayBackedTag((byte) 121, Bytes.toBytes("tagB")) }),
+        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
+            System.currentTimeMillis(), Bytes.toBytes("2"),
+            new Tag[] { new ArrayBackedTag((byte) 0, "tagA") }),
+        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes(""),
+            Bytes.toBytes("1")) };
     ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    DataOutputStream os = new DataOutputStream(byteArrayOutputStream);
-    ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(mkvA1, true));
-    KeyValueUtil.oswrite(mkvA1, os, true);
-    ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(mkvA2, true));
-    KeyValueUtil.oswrite(mkvA2, os, true);
-    DataInputStream is = new DataInputStream(new ByteArrayInputStream(
-        byteArrayOutputStream.toByteArray()));
-    KeyValue deSerKV1 = KeyValueUtil.iscreate(is, true);
-    assertTrue(kvA1.equals(deSerKV1));
-    KeyValue deSerKV2 = KeyValueUtil.iscreate(is, true);
-    assertTrue(kvA2.equals(deSerKV2));
-  }
-
-  private static class MockKeyValue implements Cell {
-    private final KeyValue kv;
-
-    public MockKeyValue(KeyValue kv) {
-      this.kv = kv;
-    }
-
-    /**
-     * This returns the offset where the tag actually starts.
-     */
-    @Override
-    public int getTagsOffset() {
-      return this.kv.getTagsOffset();
-    }
-
-    /**
-     * used to achieve atomic operations in the memstore.
-     */
-    @Override
-    public long getSequenceId() {
-      return this.kv.getSequenceId();
+    for (KeyValue kv : keyValues) {
+      DataOutputStream os = new DataOutputStream(byteArrayOutputStream);
+      ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(kv, true));
+      KeyValueUtil.oswrite(kv, os, true);
     }
-
-    /**
-     * This returns the total length of the tag bytes
-     */
-    @Override
-    public int getTagsLength() {
-      return this.kv.getTagsLength();
-    }
-
-    /**
-     *
-     * @return Timestamp
-     */
-    @Override
-    public long getTimestamp() {
-      return this.kv.getTimestamp();
-    }
-
-    /**
-     * @return KeyValue.TYPE byte representation
-     */
-    @Override
-    public byte getTypeByte() {
-      return this.kv.getTypeByte();
-    }
-
-    /**
-     * @return the backing array of the entire KeyValue (all KeyValue fields are
-     *         in a single array)
-     */
-    @Override
-    public byte[] getValueArray() {
-      return this.kv.getValueArray();
-    }
-
-    /**
-     * @return the value offset
-     */
-    @Override
-    public int getValueOffset() {
-      return this.kv.getValueOffset();
-    }
-
-    /**
-     * @return Value length
-     */
-    @Override
-    public int getValueLength() {
-      return this.kv.getValueLength();
-    }
-
-    /**
-     * @return the backing array of the entire KeyValue (all KeyValue fields are
-     *         in a single array)
-     */
-    @Override
-    public byte[] getRowArray() {
-      return this.kv.getRowArray();
-    }
-
-    /**
-     * @return Row offset
-     */
-    @Override
-    public int getRowOffset() {
-      return this.kv.getRowOffset();
-    }
-
-    /**
-     * @return Row length
-     */
-    @Override
-    public short getRowLength() {
-      return this.kv.getRowLength();
-    }
-
-    /**
-     * @return the backing array of the entire KeyValue (all KeyValue fields are
-     *         in a single array)
-     */
-    @Override
-    public byte[] getFamilyArray() {
-      return this.kv.getFamilyArray();
+    DataInputStream is =
+        new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
+    for (int i = 0; i < keyValues.length; i++) {
+      LOG.info("Case#" + i + ": deserialize the kv: " + keyValues[i]);
+      KeyValue destKv = KeyValueUtil.createKeyValueFromInputStream(is, true);
+      assertEquals(keyValues[i], destKv);
+      assertArrayEquals(CellUtil.cloneValue(keyValues[i]), CellUtil.cloneValue(destKv));
+      assertArrayEquals(PrivateCellUtil.cloneTags(keyValues[i]), PrivateCellUtil.cloneTags(destKv));
     }
+  }
 
-    /**
-     * @return Family offset
-     */
-    @Override
-    public int getFamilyOffset() {
-      return this.kv.getFamilyOffset();
+  private static class FailureCase {
+    byte[] buf;
+    int offset;
+    int length;
+    boolean withTags;
+    String expectedMessage;
+
+    public FailureCase(byte[] buf, int offset, int length, boolean withTags,
+        String expectedMessage) {
+      this.buf = buf;
+      this.offset = offset;
+      this.length = length;
+      this.withTags = withTags;
+      this.expectedMessage = expectedMessage;
     }
 
-    /**
-     * @return Family length
-     */
     @Override
-    public byte getFamilyLength() {
-      return this.kv.getFamilyLength();
+    public String toString() {
+      return "FailureCaseDetails: [buf=" + Bytes.toStringBinary(buf, offset, length) + ", offset="
+          + offset + ", " + "length=" + length + ", expectedMessage=" + expectedMessage
+          + ", withtags=" + withTags + "]";
     }
 
-    /**
-     * @return the backing array of the entire KeyValue (all KeyValue fields are
-     *         in a single array)
-     */
-    @Override
-    public byte[] getQualifierArray() {
-      return this.kv.getQualifierArray();
+    public String getExpectedMessage() {
+      return this.expectedMessage + KeyValueUtil.bytesToHex(buf, offset, length);
     }
+  }
 
-    /**
-     * @return Qualifier offset
-     */
-    @Override
-    public int getQualifierOffset() {
-      return this.kv.getQualifierOffset();
+  @Test
+  public void testCheckKeyValueBytesFailureCase() throws Exception {
+    byte[][] inputs = new byte[][] { HConstants.EMPTY_BYTE_ARRAY, // case.0
+      Bytes.toBytesBinary("a"), // case.1
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x01"), // case.2
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00"), // case.3
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01"), // case.4
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00"), // case.5
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00\\x01"), // case.6
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00\\x03ROW"), // case.7
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01"), // case.8
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\xFF"
+          + "\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\x03"), // case.9
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x03"), // case.10
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04"), // case.11
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04VALUE"), // case.12
+    };
+    String[] outputs = new String[] { "Overflow when reading key length at position=0",
+      "Overflow when reading key length at position=0",
+      "Invalid key length in KeyValue. keyLength=1",
+      "Overflow when reading value length at position=4",
+      "Invalid value length in KeyValue, valueLength=1",
+      "Overflow when reading row length at position=8",
+      "Invalid row length in KeyValue, rowLength=1",
+      "Overflow when reading family length at position=13",
+      "Invalid family length in KeyValue, familyLength=1", "Timestamp cannot be negative, ts=-1",
+      "Invalid type in KeyValue, type=3", "Overflow when reading value part at position=25",
+      "Invalid tags length in KeyValue at position=26", };
+    byte[][] withTagsInputs = new byte[][] {
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x01"), // case.13
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x01"), // case.14
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x04\\x00\\x03\\x00A"), // case.15
+      // case.16
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x0A\\x00\\x04\\x00TAG\\x00\\x04"
+          + "\\xFFT"),
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x0C\\x00\\x04\\x00TAG\\x00\\x05"
+          + "\\xF0COME\\x00"), // case.17
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x0C\\x00\\x04\\x00TAG\\x00\\x05"
+          + "\\xF0COME"), // case.18
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x00"), // case.19
+      Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
+          + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x1B\\x00\\x05\\x01TAG1\\x00\\x05"
+          + "\\x02TAG2\\x00\\x05\\x03TAG3\\x00\\x05\\x04TAG4"), // case.20
+    };
+    String[] withTagsOutputs = new String[] { "Overflow when reading tags length at position=26",
+      "Invalid tags length in KeyValue at position=26",
+      "Invalid tag length at position=28, tagLength=3",
+      "Invalid tag length at position=34, tagLength=4",
+      "Some redundant bytes in KeyValue's buffer, startOffset=41, endOffset=42", null, null,
+      null, };
+    assertEquals(inputs.length, outputs.length);
+    assertEquals(withTagsInputs.length, withTagsOutputs.length);
+
+    FailureCase[] cases = new FailureCase[inputs.length + withTagsInputs.length];
+    for (int i = 0; i < inputs.length; i++) {
+      cases[i] = new FailureCase(inputs[i], 0, inputs[i].length, false, outputs[i]);
     }
-
-    /**
-     * @return Qualifier length
-     */
-    @Override
-    public int getQualifierLength() {
-      return this.kv.getQualifierLength();
+    for (int i = 0; i < withTagsInputs.length; i++) {
+      cases[inputs.length + i] =
+          new FailureCase(withTagsInputs[i], 0, withTagsInputs[i].length, true, withTagsOutputs[i]);
     }
 
-    /**
-     * @return the backing array of the entire KeyValue (all KeyValue fields are
-     *         in a single array)
-     */
-    @Override
-    public byte[] getTagsArray() {
-      return this.kv.getTagsArray();
+    for (int i = 0; i < cases.length; i++) {
+      FailureCase c = cases[i];
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream os = new DataOutputStream(baos);
+      ByteBufferUtils.putInt(os, c.length);
+      os.write(c.buf, c.offset, c.length);
+      try {
+        KeyValueUtil.createKeyValueFromInputStream(
+          new DataInputStream(new ByteArrayInputStream(baos.toByteArray())), c.withTags);
+        if (c.expectedMessage != null) {
+          fail("Should fail when parse key value from an invalid bytes for case#" + i + ". " + c);
+        }
+      } catch (IllegalArgumentException e) {
+        assertEquals("Case#" + i + " failed," + c, c.getExpectedMessage(), e.getMessage());
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/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 b3e5b97..5056ad7 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
@@ -1710,7 +1710,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
 
   @Override
   public boolean hasReferences() {
-    List<HStoreFile> reloadedStoreFiles = null;
     // Grab the read lock here, because we need to ensure that: only when the atomic
     // replaceStoreFiles(..) finished, we can get all the complete store file list.
     this.lock.readLock().lock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
index 5766517..569ede2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
@@ -332,7 +332,7 @@ public class TestDataBlockEncoders {
     byte[] family = new byte[0];
     byte[] qualifier = new byte[0];
     byte[] value = new byte[0];
-    KeyValue expectedKV = new KeyValue(row, family, qualifier, -1L, Type.Put, value);
+    KeyValue expectedKV = new KeyValue(row, family, qualifier, 1L, Type.Put, value);
     kvList.add(expectedKV);
     DataBlockEncoding encoding = DataBlockEncoding.ROW_INDEX_V1;
     DataBlockEncoder encoder = encoding.getEncoder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/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 4163d55..115e765 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
@@ -392,11 +392,11 @@ public class TestCacheOnWrite {
         tags[0] = t;
         kv =
             new KeyValue(row, 0, row.length, cf, 0, cf.length, qualifier, 0, qualifier.length,
-                rand.nextLong(), generateKeyType(rand), value, 0, value.length, tagList);
+                Math.abs(rand.nextLong()), generateKeyType(rand), value, 0, value.length, tagList);
       } else {
         kv =
             new KeyValue(row, 0, row.length, cf, 0, cf.length, qualifier, 0, qualifier.length,
-                rand.nextLong(), generateKeyType(rand), value, 0, value.length);
+                Math.abs(rand.nextLong()), generateKeyType(rand), value, 0, value.length);
       }
       sfw.append(kv);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/12786f80/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
index 77108c9..4dc9c68 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.testclassification.IOTests;
@@ -200,15 +199,15 @@ public class TestHFileSeek extends TestCase {
     timer.start();
     for (int i = 0; i < options.seekCount; ++i) {
       kSampler.next(key);
-      byte [] k = new byte [key.getLength()];
+      byte[] k = new byte[key.getLength()];
       System.arraycopy(key.getBytes(), 0, k, 0, key.getLength());
-      if (scanner.seekTo(KeyValueUtil.createKeyValueFromKey(k)) >= 0) {
+      KeyValue kv = new KeyValue(k, CF, QUAL);
+      if (scanner.seekTo(kv) >= 0) {
         ByteBuffer bbkey = ByteBuffer.wrap(((KeyValue) scanner.getKey()).getKey());
         ByteBuffer bbval = scanner.getValue();
         totalBytes += bbkey.limit();
         totalBytes += bbval.limit();
-      }
-      else {
+      } else {
         ++miss;
       }
     }


[09/47] hbase git commit: HBASE-21578 Fix wrong throttling exception for capacity unit

Posted by zh...@apache.org.
HBASE-21578 Fix wrong throttling exception for capacity unit

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

Branch: refs/heads/HBASE-21512
Commit: 1b08ba7385d0dd914a6fb9722b786e4ece116b28
Parents: 3ff274e
Author: meiyi <my...@gamil.com>
Authored: Fri Dec 14 09:42:48 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Dec 14 18:17:47 2018 +0800

----------------------------------------------------------------------
 .../hbase/quotas/RpcThrottlingException.java    | 21 ++++++++++++++++++--
 .../hadoop/hbase/quotas/TimeBasedLimiter.java   |  8 ++++----
 2 files changed, 23 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1b08ba73/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/RpcThrottlingException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/RpcThrottlingException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/RpcThrottlingException.java
index 9baf91f..4c48f65 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/RpcThrottlingException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/RpcThrottlingException.java
@@ -29,13 +29,15 @@ public class RpcThrottlingException extends HBaseIOException {
   @InterfaceAudience.Public
   public enum Type {
     NumRequestsExceeded, RequestSizeExceeded, NumReadRequestsExceeded, NumWriteRequestsExceeded,
-    WriteSizeExceeded, ReadSizeExceeded,
+    WriteSizeExceeded, ReadSizeExceeded, RequestCapacityUnitExceeded, ReadCapacityUnitExceeded,
+    WriteCapacityUnitExceeded
   }
 
   private static final String[] MSG_TYPE =
       new String[] { "number of requests exceeded", "request size limit exceeded",
         "number of read requests exceeded", "number of write requests exceeded",
-        "write size limit exceeded", "read size limit exceeded", };
+        "write size limit exceeded", "read size limit exceeded", "request capacity unit exceeded",
+        "read capacity unit exceeded", "write capacity unit exceeded" };
 
   private static final String MSG_WAIT = " - wait ";
 
@@ -100,6 +102,21 @@ public class RpcThrottlingException extends HBaseIOException {
     throwThrottlingException(Type.ReadSizeExceeded, waitInterval);
   }
 
+  public static void throwRequestCapacityUnitExceeded(final long waitInterval)
+      throws RpcThrottlingException {
+    throwThrottlingException(Type.RequestCapacityUnitExceeded, waitInterval);
+  }
+
+  public static void throwReadCapacityUnitExceeded(final long waitInterval)
+      throws RpcThrottlingException {
+    throwThrottlingException(Type.ReadCapacityUnitExceeded, waitInterval);
+  }
+
+  public static void throwWriteCapacityUnitExceeded(final long waitInterval)
+      throws RpcThrottlingException {
+    throwThrottlingException(Type.WriteCapacityUnitExceeded, waitInterval);
+  }
+
   private static void throwThrottlingException(final Type type, final long waitInterval)
       throws RpcThrottlingException {
     String msg = MSG_TYPE[type.ordinal()] + MSG_WAIT + StringUtils.formatTime(waitInterval);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1b08ba73/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
index 771eed1..6b5349f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/TimeBasedLimiter.java
@@ -148,7 +148,7 @@ public class TimeBasedLimiter implements QuotaLimiter {
           reqSizeLimiter.waitInterval(estimateWriteSize + estimateReadSize));
     }
     if (!reqCapacityUnitLimiter.canExecute(estimateWriteCapacityUnit + estimateReadCapacityUnit)) {
-      RpcThrottlingException.throwRequestSizeExceeded(
+      RpcThrottlingException.throwRequestCapacityUnitExceeded(
         reqCapacityUnitLimiter.waitInterval(estimateWriteCapacityUnit + estimateReadCapacityUnit));
     }
 
@@ -161,7 +161,7 @@ public class TimeBasedLimiter implements QuotaLimiter {
             writeSizeLimiter.waitInterval(estimateWriteSize));
       }
       if (!writeCapacityUnitLimiter.canExecute(estimateWriteCapacityUnit)) {
-        RpcThrottlingException.throwWriteSizeExceeded(
+        RpcThrottlingException.throwWriteCapacityUnitExceeded(
           writeCapacityUnitLimiter.waitInterval(estimateWriteCapacityUnit));
       }
     }
@@ -175,8 +175,8 @@ public class TimeBasedLimiter implements QuotaLimiter {
             readSizeLimiter.waitInterval(estimateReadSize));
       }
       if (!readCapacityUnitLimiter.canExecute(estimateReadCapacityUnit)) {
-        RpcThrottlingException
-            .throwWriteSizeExceeded(readCapacityUnitLimiter.waitInterval(estimateReadCapacityUnit));
+        RpcThrottlingException.throwReadCapacityUnitExceeded(
+          readCapacityUnitLimiter.waitInterval(estimateReadCapacityUnit));
       }
     }
   }


[21/47] hbase git commit: HBASE-21610, numOpenConnections metric is set to -1 when zero server channel exist

Posted by zh...@apache.org.
HBASE-21610, numOpenConnections metric is set to -1 when zero server channel exist


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

Branch: refs/heads/HBASE-21512
Commit: 787567336afb9c5c1e00aaa0326566a5522a5e31
Parents: 8991877
Author: Pankaj <pa...@huawei.com>
Authored: Tue Dec 18 01:31:55 2018 +0530
Committer: stack <st...@apache.org>
Committed: Thu Dec 20 16:36:42 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/78756733/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
index 8ea2057..742a728 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
@@ -169,8 +169,9 @@ public class NettyRpcServer extends RpcServer {
 
   @Override
   public int getNumOpenConnections() {
+    int channelsCount = allChannels.size();
     // allChannels also contains the server channel, so exclude that from the count.
-    return allChannels.size() - 1;
+    return channelsCount > 0 ? channelsCount - 1 : channelsCount;
   }
 
   @Override


[13/47] hbase git commit: HBASE-21589 TestCleanupMetaWAL fails

Posted by zh...@apache.org.
HBASE-21589 TestCleanupMetaWAL fails


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

Branch: refs/heads/HBASE-21512
Commit: 68b5df00951d3ee55efaa6068f4530dca17eae1f
Parents: ac0b3bb
Author: stack <st...@apache.org>
Authored: Sun Dec 16 14:15:00 2018 -0800
Committer: stack <st...@apache.org>
Committed: Mon Dec 17 09:31:59 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java  | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/68b5df00/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
index 4a723c0..03b3316 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
@@ -66,10 +66,13 @@ public class TestCleanupMetaWAL {
         .getRegionServer(TEST_UTIL.getMiniHBaseCluster().getServerWithMeta());
     TEST_UTIL.getAdmin()
         .move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), null);
+    LOG.info("KILL");
     TEST_UTIL.getMiniHBaseCluster().killRegionServer(serverWithMeta.getServerName());
-    TEST_UTIL.waitFor(10000, () ->
+    LOG.info("WAIT");
+    TEST_UTIL.waitFor(30000, () ->
         TEST_UTIL.getMiniHBaseCluster().getMaster().getProcedures().stream()
             .filter(p -> p instanceof ServerCrashProcedure && p.isFinished()).count() > 0);
+    LOG.info("DONE WAITING");
     MasterFileSystem fs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Path walPath = new Path(fs.getWALRootDir(), HConstants.HREGION_LOGDIR_NAME);
     for (FileStatus status : FSUtils.listStatus(fs.getFileSystem(), walPath)) {
@@ -77,7 +80,5 @@ public class TestCleanupMetaWAL {
         fail("Should not have splitting wal dir here:" + status);
       }
     }
-
-
   }
 }


[20/47] hbase git commit: HBASE-21514: Refactor CacheConfig(addendum)

Posted by zh...@apache.org.
HBASE-21514: Refactor CacheConfig(addendum)


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

Branch: refs/heads/HBASE-21512
Commit: 8991877bb250ee1fe66c2b9a491645973927d674
Parents: fb58a23
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 18 16:46:34 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Wed Dec 19 13:55:13 2018 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/8991877b/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 0fc9576..d095ceb 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
@@ -1300,8 +1300,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
       boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType,
       DataBlockEncoding expectedDataBlockEncoding) throws IOException {
     // Check cache for block. If found return.
-    if (cacheConf.getBlockCache().isPresent()) {
-      BlockCache cache = cacheConf.getBlockCache().get();
+    BlockCache cache = cacheConf.getBlockCache().orElse(null);
+    if (cache != null) {
       HFileBlock cachedBlock =
           (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock, updateCacheMetrics);
       if (cachedBlock != null) {


[12/47] hbase git commit: HBASE-21520 TestMultiColumnScanner cost long time when using ROWCOL bloom type

Posted by zh...@apache.org.
HBASE-21520 TestMultiColumnScanner cost long time when using ROWCOL bloom type


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

Branch: refs/heads/HBASE-21512
Commit: ac0b3bb5477612cb8844c4ef10fa2be0f1d1a025
Parents: 4911534
Author: huzheng <op...@gmail.com>
Authored: Thu Dec 13 15:04:12 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Sat Dec 15 21:08:52 2018 +0800

----------------------------------------------------------------------
 .../regionserver/TestMultiColumnScanner.java    | 94 ++++++--------------
 ...olumnScannerWithAlgoGZAndNoDataEncoding.java | 48 ++++++++++
 ...lumnScannerWithAlgoGZAndUseDataEncoding.java | 48 ++++++++++
 ...iColumnScannerWithNoneAndNoDataEncoding.java | 48 ++++++++++
 ...ColumnScannerWithNoneAndUseDataEncoding.java | 48 ++++++++++
 5 files changed, 219 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ac0b3bb5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
index 2ff0d8c..bb97c9c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
@@ -32,11 +32,9 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
-import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
@@ -47,29 +45,27 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.ClassRule;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.runners.Parameterized.Parameter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Tests optimized scanning of multiple columns.
+ * Tests optimized scanning of multiple columns. <br>
+ * We separated the big test into several sub-class UT, because When in ROWCOL bloom type, we will
+ * test the row-col bloom filter frequently for saving HDFS seek once we switch from one column to
+ * another in our UT. It's cpu time consuming (~45s for each case), so moved the ROWCOL case into a
+ * separated LargeTests to avoid timeout failure. <br>
+ * <br>
+ * To be clear: In TestMultiColumnScanner, we will flush 10 (NUM_FLUSHES=10) HFiles here, and the
+ * table will put ~1000 cells (rows=20, ts=6, qualifiers=8, total=20*6*8 ~ 1000) . Each full table
+ * scan will check the ROWCOL bloom filter 20 (rows)* 8 (column) * 10 (hfiles)= 1600 times, beside
+ * it will scan the full table 6*2^8=1536 times, so finally will have 1600*1536=2457600 bloom filter
+ * testing. (See HBASE-21520)
  */
-@RunWith(Parameterized.class)
-@Category({RegionServerTests.class, MediumTests.class})
-public class TestMultiColumnScanner {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMultiColumnScanner.class);
+public abstract class TestMultiColumnScanner {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestMultiColumnScanner.class);
 
@@ -104,20 +100,19 @@ public class TestMultiColumnScanner {
   /** The probability that a column is skipped in a store file. */
   private static final double COLUMN_SKIP_IN_STORE_FILE_PROB = 0.7;
 
-  /** The probability of skipping a column in a single row */
-  private static final double COLUMN_SKIP_IN_ROW_PROB = 0.1;
-
-  /** The probability of skipping a column everywhere */
-  private static final double COLUMN_SKIP_EVERYWHERE_PROB = 0.1;
-
   /** The probability to delete a row/column pair */
   private static final double DELETE_PROBABILITY = 0.02;
 
   private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
-  private final Compression.Algorithm comprAlgo;
-  private final BloomType bloomType;
-  private final DataBlockEncoding dataBlockEncoding;
+  @Parameter(0)
+  public Compression.Algorithm comprAlgo;
+
+  @Parameter(1)
+  public BloomType bloomType;
+
+  @Parameter(2)
+  public DataBlockEncoding dataBlockEncoding;
 
   // Some static sanity-checking.
   static {
@@ -128,27 +123,17 @@ public class TestMultiColumnScanner {
       assertTrue(TIMESTAMPS[i] < TIMESTAMPS[i + 1]);
   }
 
-  @Parameters
-  public static final Collection<Object[]> parameters() {
+  public static Collection<Object[]> generateParams(Compression.Algorithm algo,
+      boolean useDataBlockEncoding) {
     List<Object[]> parameters = new ArrayList<>();
-    for (Object[] bloomAndCompressionParams :
-        HBaseTestingUtility.BLOOM_AND_COMPRESSION_COMBINATIONS) {
-      for (boolean useDataBlockEncoding : new boolean[]{false, true}) {
-        parameters.add(ArrayUtils.add(bloomAndCompressionParams,
-            useDataBlockEncoding));
-      }
+    for (BloomType bloomType : BloomType.values()) {
+      DataBlockEncoding dataBlockEncoding =
+          useDataBlockEncoding ? DataBlockEncoding.PREFIX : DataBlockEncoding.NONE;
+      parameters.add(new Object[] { algo, bloomType, dataBlockEncoding });
     }
     return parameters;
   }
 
-  public TestMultiColumnScanner(Compression.Algorithm comprAlgo,
-      BloomType bloomType, boolean useDataBlockEncoding) {
-    this.comprAlgo = comprAlgo;
-    this.bloomType = bloomType;
-    this.dataBlockEncoding = useDataBlockEncoding ? DataBlockEncoding.PREFIX :
-        DataBlockEncoding.NONE;
-  }
-
   @Test
   public void testMultiColumnScanner() throws IOException {
     TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
@@ -170,24 +155,6 @@ public class TestMultiColumnScanner {
     Map<String, Long> lastDelTimeMap = new HashMap<>();
 
     Random rand = new Random(29372937L);
-    Set<String> rowQualSkip = new HashSet<>();
-
-    // Skip some columns in some rows. We need to test scanning over a set
-    // of columns when some of the columns are not there.
-    for (String row : rows)
-      for (String qual : qualifiers)
-        if (rand.nextDouble() < COLUMN_SKIP_IN_ROW_PROB) {
-          LOG.info("Skipping " + qual + " in row " + row);
-          rowQualSkip.add(rowQualKey(row, qual));
-        }
-
-    // Also skip some columns in all rows.
-    for (String qual : qualifiers)
-      if (rand.nextDouble() < COLUMN_SKIP_EVERYWHERE_PROB) {
-        LOG.info("Skipping " + qual + " in all rows");
-        for (String row : rows)
-          rowQualSkip.add(rowQualKey(row, qual));
-      }
 
     for (int iFlush = 0; iFlush < NUM_FLUSHES; ++iFlush) {
       for (String qual : qualifiers) {
@@ -316,10 +283,6 @@ public class TestMultiColumnScanner {
         kv.getQualifierLength());
   }
 
-  private static String rowQualKey(String row, String qual) {
-    return row + "_" + qual;
-  }
-
   static String createValue(String row, String qual, long ts) {
     return "value_for_" + row + "_" + qual + "_" + ts;
   }
@@ -339,10 +302,7 @@ public class TestMultiColumnScanner {
 
       lst.add(sb.toString());
     }
-
     return lst;
   }
-
-
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac0b3bb5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndNoDataEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndNoDataEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndNoDataEncoding.java
new file mode 100644
index 0000000..cc68c11
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndNoDataEncoding.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.util.Collection;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Test case for Compression.Algorithm.GZ and no use data block encoding.
+ * @see org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner
+ */
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, LargeTests.class })
+public class TestMultiColumnScannerWithAlgoGZAndNoDataEncoding extends TestMultiColumnScanner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMultiColumnScannerWithAlgoGZAndNoDataEncoding.class);
+
+  @Parameters
+  public static Collection<Object[]> parameters() {
+    return TestMultiColumnScanner.generateParams(Algorithm.GZ, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac0b3bb5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndUseDataEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndUseDataEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndUseDataEncoding.java
new file mode 100644
index 0000000..c817da2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithAlgoGZAndUseDataEncoding.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.util.Collection;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Test case for Compression.Algorithm.GZ and use data block encoding.
+ * @see org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner
+ */
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, LargeTests.class })
+public class TestMultiColumnScannerWithAlgoGZAndUseDataEncoding extends TestMultiColumnScanner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMultiColumnScannerWithAlgoGZAndUseDataEncoding.class);
+
+  @Parameters
+  public static Collection<Object[]> parameters() {
+    return TestMultiColumnScanner.generateParams(Algorithm.GZ, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac0b3bb5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndNoDataEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndNoDataEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndNoDataEncoding.java
new file mode 100644
index 0000000..4f6aa90
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndNoDataEncoding.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.util.Collection;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Test case for Compression.Algorithm.NONE and no use data block encoding.
+ * @see org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner
+ */
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, LargeTests.class })
+public class TestMultiColumnScannerWithNoneAndNoDataEncoding extends TestMultiColumnScanner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMultiColumnScannerWithNoneAndNoDataEncoding.class);
+
+  @Parameters
+  public static Collection<Object[]> parameters() {
+    return TestMultiColumnScanner.generateParams(Algorithm.NONE, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac0b3bb5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndUseDataEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndUseDataEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndUseDataEncoding.java
new file mode 100644
index 0000000..f1fd30d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScannerWithNoneAndUseDataEncoding.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.util.Collection;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Test case for Compression.Algorithm.NONE and no use data block encoding.
+ * @see org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner
+ */
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, LargeTests.class })
+public class TestMultiColumnScannerWithNoneAndUseDataEncoding extends TestMultiColumnScanner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMultiColumnScannerWithNoneAndUseDataEncoding.class);
+
+  @Parameters
+  public static Collection<Object[]> parameters() {
+    return TestMultiColumnScanner.generateParams(Algorithm.NONE, true);
+  }
+}


[38/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
index 2fb3f76..0f27519 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Represents a single cell and the amount to increment it by
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
index 3ceb4c0..6cded1b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Represents a single cell and its value.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue");
 
@@ -46,6 +46,7 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
   private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (short)4);
   private static final org.apache.thrift.protocol.TField TAGS_FIELD_DESC = new org.apache.thrift.protocol.TField("tags", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.BYTE, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -58,6 +59,7 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
   public ByteBuffer value; // required
   public long timestamp; // optional
   public ByteBuffer tags; // optional
+  public byte type; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -65,7 +67,8 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
     QUALIFIER((short)2, "qualifier"),
     VALUE((short)3, "value"),
     TIMESTAMP((short)4, "timestamp"),
-    TAGS((short)5, "tags");
+    TAGS((short)5, "tags"),
+    TYPE((short)6, "type");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -90,6 +93,8 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
           return TIMESTAMP;
         case 5: // TAGS
           return TAGS;
+        case 6: // TYPE
+          return TYPE;
         default:
           return null;
       }
@@ -131,8 +136,9 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
 
   // isset id assignments
   private static final int __TIMESTAMP_ISSET_ID = 0;
+  private static final int __TYPE_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.TIMESTAMP,_Fields.TAGS};
+  private static final _Fields optionals[] = {_Fields.TIMESTAMP,_Fields.TAGS,_Fields.TYPE};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -146,6 +152,8 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.TAGS, new org.apache.thrift.meta_data.FieldMetaData("tags", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnValue.class, metaDataMap);
   }
@@ -182,6 +190,7 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
     if (other.isSetTags()) {
       this.tags = org.apache.thrift.TBaseHelper.copyBinary(other.tags);
     }
+    this.type = other.type;
   }
 
   public TColumnValue deepCopy() {
@@ -196,6 +205,8 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
     setTimestampIsSet(false);
     this.timestamp = 0;
     this.tags = null;
+    setTypeIsSet(false);
+    this.type = 0;
   }
 
   public byte[] getFamily() {
@@ -357,6 +368,29 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
     }
   }
 
+  public byte getType() {
+    return this.type;
+  }
+
+  public TColumnValue setType(byte type) {
+    this.type = type;
+    setTypeIsSet(true);
+    return this;
+  }
+
+  public void unsetType() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TYPE_ISSET_ID);
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean isSetType() {
+    return EncodingUtils.testBit(__isset_bitfield, __TYPE_ISSET_ID);
+  }
+
+  public void setTypeIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TYPE_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case FAMILY:
@@ -399,6 +433,14 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
       }
       break;
 
+    case TYPE:
+      if (value == null) {
+        unsetType();
+      } else {
+        setType((Byte)value);
+      }
+      break;
+
     }
   }
 
@@ -419,6 +461,9 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
     case TAGS:
       return getTags();
 
+    case TYPE:
+      return getType();
+
     }
     throw new IllegalStateException();
   }
@@ -440,6 +485,8 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
       return isSetTimestamp();
     case TAGS:
       return isSetTags();
+    case TYPE:
+      return isSetType();
     }
     throw new IllegalStateException();
   }
@@ -502,6 +549,15 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
         return false;
     }
 
+    boolean this_present_type = true && this.isSetType();
+    boolean that_present_type = true && that.isSetType();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (this.type != that.type)
+        return false;
+    }
+
     return true;
   }
 
@@ -534,6 +590,11 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
     if (present_tags)
       list.add(tags);
 
+    boolean present_type = true && (isSetType());
+    list.add(present_type);
+    if (present_type)
+      list.add(type);
+
     return list.hashCode();
   }
 
@@ -595,6 +656,16 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetType()).compareTo(other.isSetType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -654,6 +725,12 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
       }
       first = false;
     }
+    if (isSetType()) {
+      if (!first) sb.append(", ");
+      sb.append("type:");
+      sb.append(this.type);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -748,6 +825,14 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) {
+              struct.type = iprot.readByte();
+              struct.setTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -790,6 +875,11 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
           oprot.writeFieldEnd();
         }
       }
+      if (struct.isSetType()) {
+        oprot.writeFieldBegin(TYPE_FIELD_DESC);
+        oprot.writeByte(struct.type);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -817,13 +907,19 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
       if (struct.isSetTags()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetType()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetTimestamp()) {
         oprot.writeI64(struct.timestamp);
       }
       if (struct.isSetTags()) {
         oprot.writeBinary(struct.tags);
       }
+      if (struct.isSetType()) {
+        oprot.writeByte(struct.type);
+      }
     }
 
     @Override
@@ -835,7 +931,7 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
       struct.setQualifierIsSet(true);
       struct.value = iprot.readBinary();
       struct.setValueIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.timestamp = iprot.readI64();
         struct.setTimestampIsSet(true);
@@ -844,6 +940,10 @@ public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColu
         struct.tags = iprot.readBinary();
         struct.setTagsIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.type = iprot.readByte();
+        struct.setTypeIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
new file mode 100644
index 0000000..46799be
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
@@ -0,0 +1,60 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum TCompressionAlgorithm implements org.apache.thrift.TEnum {
+  LZO(0),
+  GZ(1),
+  NONE(2),
+  SNAPPY(3),
+  LZ4(4),
+  BZIP2(5),
+  ZSTD(6);
+
+  private final int value;
+
+  private TCompressionAlgorithm(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TCompressionAlgorithm findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return LZO;
+      case 1:
+        return GZ;
+      case 2:
+        return NONE;
+      case 3:
+        return SNAPPY;
+      case 4:
+        return LZ4;
+      case 5:
+        return BZIP2;
+      case 6:
+        return ZSTD;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
new file mode 100644
index 0000000..1594aac3
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
@@ -0,0 +1,57 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum TDataBlockEncoding implements org.apache.thrift.TEnum {
+  /**
+   * Disable data block encoding.
+   */
+  NONE(0),
+  PREFIX(2),
+  DIFF(3),
+  FAST_DIFF(4),
+  ROW_INDEX_V1(7);
+
+  private final int value;
+
+  private TDataBlockEncoding(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TDataBlockEncoding findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return NONE;
+      case 2:
+        return PREFIX;
+      case 3:
+        return DIFF;
+      case 4:
+        return FAST_DIFF;
+      case 7:
+        return ROW_INDEX_V1;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
index 8f7f119..fb4ffe6 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
@@ -60,7 +60,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields>, java.io.Serializable, Cloneable, Comparable<TDelete> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
index c1144f3..dbe7393 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java
@@ -19,6 +19,7 @@ import org.apache.thrift.TEnum;
  *  - FSYNC_WAL means Write the Mutation to the WAL synchronously and force the entries to disk.
  */
 public enum TDurability implements org.apache.thrift.TEnum {
+  USE_DEFAULT(0),
   SKIP_WAL(1),
   ASYNC_WAL(2),
   SYNC_WAL(3),
@@ -43,6 +44,8 @@ public enum TDurability implements org.apache.thrift.TEnum {
    */
   public static TDurability findByValue(int value) { 
     switch (value) {
+      case 0:
+        return USE_DEFAULT;
       case 1:
         return SKIP_WAL;
       case 2:

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
index cfb1d6c..1b01ef5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
  * If you specify a time range and a timestamp the range is ignored.
  * Timestamps on TColumns are ignored.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-07-03")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.io.Serializable, Cloneable, Comparable<TGet> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet");
 
@@ -61,6 +61,10 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
   private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.STRUCT, (short)8);
   private static final org.apache.thrift.protocol.TField CONSISTENCY_FIELD_DESC = new org.apache.thrift.protocol.TField("consistency", org.apache.thrift.protocol.TType.I32, (short)9);
   private static final org.apache.thrift.protocol.TField TARGET_REPLICA_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("targetReplicaId", org.apache.thrift.protocol.TType.I32, (short)10);
+  private static final org.apache.thrift.protocol.TField CACHE_BLOCKS_FIELD_DESC = new org.apache.thrift.protocol.TField("cacheBlocks", org.apache.thrift.protocol.TType.BOOL, (short)11);
+  private static final org.apache.thrift.protocol.TField STORE_LIMIT_FIELD_DESC = new org.apache.thrift.protocol.TField("storeLimit", org.apache.thrift.protocol.TType.I32, (short)12);
+  private static final org.apache.thrift.protocol.TField STORE_OFFSET_FIELD_DESC = new org.apache.thrift.protocol.TField("storeOffset", org.apache.thrift.protocol.TType.I32, (short)13);
+  private static final org.apache.thrift.protocol.TField EXISTENCE_ONLY_FIELD_DESC = new org.apache.thrift.protocol.TField("existence_only", org.apache.thrift.protocol.TType.BOOL, (short)14);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -82,6 +86,10 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
    */
   public TConsistency consistency; // optional
   public int targetReplicaId; // optional
+  public boolean cacheBlocks; // optional
+  public int storeLimit; // optional
+  public int storeOffset; // optional
+  public boolean existence_only; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -98,7 +106,11 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
      * @see TConsistency
      */
     CONSISTENCY((short)9, "consistency"),
-    TARGET_REPLICA_ID((short)10, "targetReplicaId");
+    TARGET_REPLICA_ID((short)10, "targetReplicaId"),
+    CACHE_BLOCKS((short)11, "cacheBlocks"),
+    STORE_LIMIT((short)12, "storeLimit"),
+    STORE_OFFSET((short)13, "storeOffset"),
+    EXISTENCE_ONLY((short)14, "existence_only");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -133,6 +145,14 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
           return CONSISTENCY;
         case 10: // TARGET_REPLICA_ID
           return TARGET_REPLICA_ID;
+        case 11: // CACHE_BLOCKS
+          return CACHE_BLOCKS;
+        case 12: // STORE_LIMIT
+          return STORE_LIMIT;
+        case 13: // STORE_OFFSET
+          return STORE_OFFSET;
+        case 14: // EXISTENCE_ONLY
+          return EXISTENCE_ONLY;
         default:
           return null;
       }
@@ -176,8 +196,12 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
   private static final int __TIMESTAMP_ISSET_ID = 0;
   private static final int __MAXVERSIONS_ISSET_ID = 1;
   private static final int __TARGETREPLICAID_ISSET_ID = 2;
+  private static final int __CACHEBLOCKS_ISSET_ID = 3;
+  private static final int __STORELIMIT_ISSET_ID = 4;
+  private static final int __STOREOFFSET_ISSET_ID = 5;
+  private static final int __EXISTENCE_ONLY_ISSET_ID = 6;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.COLUMNS,_Fields.TIMESTAMP,_Fields.TIME_RANGE,_Fields.MAX_VERSIONS,_Fields.FILTER_STRING,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.CONSISTENCY,_Fields.TARGET_REPLICA_ID};
+  private static final _Fields optionals[] = {_Fields.COLUMNS,_Fields.TIMESTAMP,_Fields.TIME_RANGE,_Fields.MAX_VERSIONS,_Fields.FILTER_STRING,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.CONSISTENCY,_Fields.TARGET_REPLICA_ID,_Fields.CACHE_BLOCKS,_Fields.STORE_LIMIT,_Fields.STORE_OFFSET,_Fields.EXISTENCE_ONLY};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -204,6 +228,14 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TConsistency.class)));
     tmpMap.put(_Fields.TARGET_REPLICA_ID, new org.apache.thrift.meta_data.FieldMetaData("targetReplicaId", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.CACHE_BLOCKS, new org.apache.thrift.meta_data.FieldMetaData("cacheBlocks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.STORE_LIMIT, new org.apache.thrift.meta_data.FieldMetaData("storeLimit", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.STORE_OFFSET, new org.apache.thrift.meta_data.FieldMetaData("storeOffset", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.EXISTENCE_ONLY, new org.apache.thrift.meta_data.FieldMetaData("existence_only", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGet.class, metaDataMap);
   }
@@ -252,6 +284,10 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
       this.consistency = other.consistency;
     }
     this.targetReplicaId = other.targetReplicaId;
+    this.cacheBlocks = other.cacheBlocks;
+    this.storeLimit = other.storeLimit;
+    this.storeOffset = other.storeOffset;
+    this.existence_only = other.existence_only;
   }
 
   public TGet deepCopy() {
@@ -273,6 +309,14 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
     this.consistency = null;
     setTargetReplicaIdIsSet(false);
     this.targetReplicaId = 0;
+    setCacheBlocksIsSet(false);
+    this.cacheBlocks = false;
+    setStoreLimitIsSet(false);
+    this.storeLimit = 0;
+    setStoreOffsetIsSet(false);
+    this.storeOffset = 0;
+    setExistence_onlyIsSet(false);
+    this.existence_only = false;
   }
 
   public byte[] getRow() {
@@ -566,6 +610,98 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TARGETREPLICAID_ISSET_ID, value);
   }
 
+  public boolean isCacheBlocks() {
+    return this.cacheBlocks;
+  }
+
+  public TGet setCacheBlocks(boolean cacheBlocks) {
+    this.cacheBlocks = cacheBlocks;
+    setCacheBlocksIsSet(true);
+    return this;
+  }
+
+  public void unsetCacheBlocks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID);
+  }
+
+  /** Returns true if field cacheBlocks is set (has been assigned a value) and false otherwise */
+  public boolean isSetCacheBlocks() {
+    return EncodingUtils.testBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID);
+  }
+
+  public void setCacheBlocksIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID, value);
+  }
+
+  public int getStoreLimit() {
+    return this.storeLimit;
+  }
+
+  public TGet setStoreLimit(int storeLimit) {
+    this.storeLimit = storeLimit;
+    setStoreLimitIsSet(true);
+    return this;
+  }
+
+  public void unsetStoreLimit() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORELIMIT_ISSET_ID);
+  }
+
+  /** Returns true if field storeLimit is set (has been assigned a value) and false otherwise */
+  public boolean isSetStoreLimit() {
+    return EncodingUtils.testBit(__isset_bitfield, __STORELIMIT_ISSET_ID);
+  }
+
+  public void setStoreLimitIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORELIMIT_ISSET_ID, value);
+  }
+
+  public int getStoreOffset() {
+    return this.storeOffset;
+  }
+
+  public TGet setStoreOffset(int storeOffset) {
+    this.storeOffset = storeOffset;
+    setStoreOffsetIsSet(true);
+    return this;
+  }
+
+  public void unsetStoreOffset() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STOREOFFSET_ISSET_ID);
+  }
+
+  /** Returns true if field storeOffset is set (has been assigned a value) and false otherwise */
+  public boolean isSetStoreOffset() {
+    return EncodingUtils.testBit(__isset_bitfield, __STOREOFFSET_ISSET_ID);
+  }
+
+  public void setStoreOffsetIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STOREOFFSET_ISSET_ID, value);
+  }
+
+  public boolean isExistence_only() {
+    return this.existence_only;
+  }
+
+  public TGet setExistence_only(boolean existence_only) {
+    this.existence_only = existence_only;
+    setExistence_onlyIsSet(true);
+    return this;
+  }
+
+  public void unsetExistence_only() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EXISTENCE_ONLY_ISSET_ID);
+  }
+
+  /** Returns true if field existence_only is set (has been assigned a value) and false otherwise */
+  public boolean isSetExistence_only() {
+    return EncodingUtils.testBit(__isset_bitfield, __EXISTENCE_ONLY_ISSET_ID);
+  }
+
+  public void setExistence_onlyIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXISTENCE_ONLY_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ROW:
@@ -648,6 +784,38 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
       }
       break;
 
+    case CACHE_BLOCKS:
+      if (value == null) {
+        unsetCacheBlocks();
+      } else {
+        setCacheBlocks((Boolean)value);
+      }
+      break;
+
+    case STORE_LIMIT:
+      if (value == null) {
+        unsetStoreLimit();
+      } else {
+        setStoreLimit((Integer)value);
+      }
+      break;
+
+    case STORE_OFFSET:
+      if (value == null) {
+        unsetStoreOffset();
+      } else {
+        setStoreOffset((Integer)value);
+      }
+      break;
+
+    case EXISTENCE_ONLY:
+      if (value == null) {
+        unsetExistence_only();
+      } else {
+        setExistence_only((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -683,6 +851,18 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
     case TARGET_REPLICA_ID:
       return getTargetReplicaId();
 
+    case CACHE_BLOCKS:
+      return isCacheBlocks();
+
+    case STORE_LIMIT:
+      return getStoreLimit();
+
+    case STORE_OFFSET:
+      return getStoreOffset();
+
+    case EXISTENCE_ONLY:
+      return isExistence_only();
+
     }
     throw new IllegalStateException();
   }
@@ -714,6 +894,14 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
       return isSetConsistency();
     case TARGET_REPLICA_ID:
       return isSetTargetReplicaId();
+    case CACHE_BLOCKS:
+      return isSetCacheBlocks();
+    case STORE_LIMIT:
+      return isSetStoreLimit();
+    case STORE_OFFSET:
+      return isSetStoreOffset();
+    case EXISTENCE_ONLY:
+      return isSetExistence_only();
     }
     throw new IllegalStateException();
   }
@@ -821,6 +1009,42 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
         return false;
     }
 
+    boolean this_present_cacheBlocks = true && this.isSetCacheBlocks();
+    boolean that_present_cacheBlocks = true && that.isSetCacheBlocks();
+    if (this_present_cacheBlocks || that_present_cacheBlocks) {
+      if (!(this_present_cacheBlocks && that_present_cacheBlocks))
+        return false;
+      if (this.cacheBlocks != that.cacheBlocks)
+        return false;
+    }
+
+    boolean this_present_storeLimit = true && this.isSetStoreLimit();
+    boolean that_present_storeLimit = true && that.isSetStoreLimit();
+    if (this_present_storeLimit || that_present_storeLimit) {
+      if (!(this_present_storeLimit && that_present_storeLimit))
+        return false;
+      if (this.storeLimit != that.storeLimit)
+        return false;
+    }
+
+    boolean this_present_storeOffset = true && this.isSetStoreOffset();
+    boolean that_present_storeOffset = true && that.isSetStoreOffset();
+    if (this_present_storeOffset || that_present_storeOffset) {
+      if (!(this_present_storeOffset && that_present_storeOffset))
+        return false;
+      if (this.storeOffset != that.storeOffset)
+        return false;
+    }
+
+    boolean this_present_existence_only = true && this.isSetExistence_only();
+    boolean that_present_existence_only = true && that.isSetExistence_only();
+    if (this_present_existence_only || that_present_existence_only) {
+      if (!(this_present_existence_only && that_present_existence_only))
+        return false;
+      if (this.existence_only != that.existence_only)
+        return false;
+    }
+
     return true;
   }
 
@@ -878,6 +1102,26 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
     if (present_targetReplicaId)
       list.add(targetReplicaId);
 
+    boolean present_cacheBlocks = true && (isSetCacheBlocks());
+    list.add(present_cacheBlocks);
+    if (present_cacheBlocks)
+      list.add(cacheBlocks);
+
+    boolean present_storeLimit = true && (isSetStoreLimit());
+    list.add(present_storeLimit);
+    if (present_storeLimit)
+      list.add(storeLimit);
+
+    boolean present_storeOffset = true && (isSetStoreOffset());
+    list.add(present_storeOffset);
+    if (present_storeOffset)
+      list.add(storeOffset);
+
+    boolean present_existence_only = true && (isSetExistence_only());
+    list.add(present_existence_only);
+    if (present_existence_only)
+      list.add(existence_only);
+
     return list.hashCode();
   }
 
@@ -989,6 +1233,46 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCacheBlocks()).compareTo(other.isSetCacheBlocks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCacheBlocks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cacheBlocks, other.cacheBlocks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetStoreLimit()).compareTo(other.isSetStoreLimit());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetStoreLimit()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storeLimit, other.storeLimit);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetStoreOffset()).compareTo(other.isSetStoreOffset());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetStoreOffset()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storeOffset, other.storeOffset);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetExistence_only()).compareTo(other.isSetExistence_only());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetExistence_only()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.existence_only, other.existence_only);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1094,6 +1378,30 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
       sb.append(this.targetReplicaId);
       first = false;
     }
+    if (isSetCacheBlocks()) {
+      if (!first) sb.append(", ");
+      sb.append("cacheBlocks:");
+      sb.append(this.cacheBlocks);
+      first = false;
+    }
+    if (isSetStoreLimit()) {
+      if (!first) sb.append(", ");
+      sb.append("storeLimit:");
+      sb.append(this.storeLimit);
+      first = false;
+    }
+    if (isSetStoreOffset()) {
+      if (!first) sb.append(", ");
+      sb.append("storeOffset:");
+      sb.append(this.storeOffset);
+      first = false;
+    }
+    if (isSetExistence_only()) {
+      if (!first) sb.append(", ");
+      sb.append("existence_only:");
+      sb.append(this.existence_only);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1253,6 +1561,38 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 11: // CACHE_BLOCKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.cacheBlocks = iprot.readBool();
+              struct.setCacheBlocksIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // STORE_LIMIT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.storeLimit = iprot.readI32();
+              struct.setStoreLimitIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // STORE_OFFSET
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.storeOffset = iprot.readI32();
+              struct.setStoreOffsetIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 14: // EXISTENCE_ONLY
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.existence_only = iprot.readBool();
+              struct.setExistence_onlyIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1345,6 +1685,26 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
         oprot.writeI32(struct.targetReplicaId);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetCacheBlocks()) {
+        oprot.writeFieldBegin(CACHE_BLOCKS_FIELD_DESC);
+        oprot.writeBool(struct.cacheBlocks);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetStoreLimit()) {
+        oprot.writeFieldBegin(STORE_LIMIT_FIELD_DESC);
+        oprot.writeI32(struct.storeLimit);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetStoreOffset()) {
+        oprot.writeFieldBegin(STORE_OFFSET_FIELD_DESC);
+        oprot.writeI32(struct.storeOffset);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetExistence_only()) {
+        oprot.writeFieldBegin(EXISTENCE_ONLY_FIELD_DESC);
+        oprot.writeBool(struct.existence_only);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1391,7 +1751,19 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
       if (struct.isSetTargetReplicaId()) {
         optionals.set(8);
       }
-      oprot.writeBitSet(optionals, 9);
+      if (struct.isSetCacheBlocks()) {
+        optionals.set(9);
+      }
+      if (struct.isSetStoreLimit()) {
+        optionals.set(10);
+      }
+      if (struct.isSetStoreOffset()) {
+        optionals.set(11);
+      }
+      if (struct.isSetExistence_only()) {
+        optionals.set(12);
+      }
+      oprot.writeBitSet(optionals, 13);
       if (struct.isSetColumns()) {
         {
           oprot.writeI32(struct.columns.size());
@@ -1432,6 +1804,18 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
       if (struct.isSetTargetReplicaId()) {
         oprot.writeI32(struct.targetReplicaId);
       }
+      if (struct.isSetCacheBlocks()) {
+        oprot.writeBool(struct.cacheBlocks);
+      }
+      if (struct.isSetStoreLimit()) {
+        oprot.writeI32(struct.storeLimit);
+      }
+      if (struct.isSetStoreOffset()) {
+        oprot.writeI32(struct.storeOffset);
+      }
+      if (struct.isSetExistence_only()) {
+        oprot.writeBool(struct.existence_only);
+      }
     }
 
     @Override
@@ -1439,7 +1823,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.row = iprot.readBinary();
       struct.setRowIsSet(true);
-      BitSet incoming = iprot.readBitSet(9);
+      BitSet incoming = iprot.readBitSet(13);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TList _list27 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
@@ -1499,6 +1883,22 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
         struct.targetReplicaId = iprot.readI32();
         struct.setTargetReplicaIdIsSet(true);
       }
+      if (incoming.get(9)) {
+        struct.cacheBlocks = iprot.readBool();
+        struct.setCacheBlocksIsSet(true);
+      }
+      if (incoming.get(10)) {
+        struct.storeLimit = iprot.readI32();
+        struct.setStoreLimitIsSet(true);
+      }
+      if (incoming.get(11)) {
+        struct.storeOffset = iprot.readI32();
+        struct.setStoreOffsetIsSet(true);
+      }
+      if (incoming.get(12)) {
+        struct.existence_only = iprot.readBool();
+        struct.setExistence_onlyIsSet(true);
+      }
     }
   }
 


[24/47] hbase git commit: HBASE-21620 Problem in scan query when using more than one column prefix filter in some cases

Posted by zh...@apache.org.
HBASE-21620 Problem in scan query when using more than one column prefix filter in some cases

Signed-off-by: Guanghao Zhang <zg...@apache.org>
Signed-off-by: Michael Stack <st...@apache.org>
Signed-off-by: Allan Yang <al...@apache.org>


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

Branch: refs/heads/HBASE-21512
Commit: e160b5ac8d82330911ea746e456ea53bf317ace8
Parents: 12786f8
Author: openinx <op...@gmail.com>
Authored: Thu Dec 20 21:04:10 2018 +0800
Committer: stack <st...@apache.org>
Committed: Fri Dec 21 15:21:53 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/FilterListWithOR.java   | 65 ++++++++++----------
 .../hadoop/hbase/regionserver/StoreScanner.java |  2 +-
 .../hadoop/hbase/filter/TestFilterList.java     | 62 +++++++++++++++++--
 .../hbase/filter/TestFilterListOnMini.java      | 50 ++++++++++++++-
 4 files changed, 140 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e160b5ac/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
index 842fdc5..ba4cd88 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
@@ -83,30 +83,40 @@ public class FilterListWithOR extends FilterListBase {
    * next family for RegionScanner, INCLUDE_AND_NEXT_ROW is the same. so we should pass current cell
    * to the filter, if row mismatch or row match but column family mismatch. (HBASE-18368)
    * @see org.apache.hadoop.hbase.filter.Filter.ReturnCode
+   * @param subFilter which sub-filter to calculate the return code by using previous cell and
+   *          previous return code.
+   * @param prevCell the previous cell passed to given sub-filter.
+   * @param currentCell the current cell which will pass to given sub-filter.
+   * @param prevCode the previous return code for given sub-filter.
+   * @return return code calculated by using previous cell and previous return code. null means can
+   *         not decide which return code should return, so we will pass the currentCell to
+   *         subFilter for getting currentCell's return code, and it won't impact the sub-filter's
+   *         internal states.
    */
-  private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell currentCell,
-      ReturnCode prevCode) throws IOException {
+  private ReturnCode calculateReturnCodeByPrevCellAndRC(Filter subFilter, Cell currentCell,
+      Cell prevCell, ReturnCode prevCode) throws IOException {
     if (prevCell == null || prevCode == null) {
-      return true;
+      return null;
     }
     switch (prevCode) {
     case INCLUDE:
     case SKIP:
-      return true;
+        return null;
     case SEEK_NEXT_USING_HINT:
-      Cell nextHintCell = getNextCellHint(prevCell);
-      return nextHintCell == null || this.compareCell(currentCell, nextHintCell) >= 0;
+        Cell nextHintCell = subFilter.getNextCellHint(prevCell);
+        return nextHintCell != null && compareCell(currentCell, nextHintCell) < 0
+          ? ReturnCode.SEEK_NEXT_USING_HINT : null;
     case NEXT_COL:
     case INCLUDE_AND_NEXT_COL:
-      // Once row changed, reset() will clear prevCells, so we need not to compare their rows
-      // because rows are the same here.
-      return !CellUtil.matchingColumn(prevCell, currentCell);
+        // Once row changed, reset() will clear prevCells, so we need not to compare their rows
+        // because rows are the same here.
+        return CellUtil.matchingColumn(prevCell, currentCell) ? ReturnCode.NEXT_COL : null;
     case NEXT_ROW:
     case INCLUDE_AND_SEEK_NEXT_ROW:
-      // As described above, rows are definitely the same, so we only compare the family.
-      return !CellUtil.matchingFamily(prevCell, currentCell);
+        // As described above, rows are definitely the same, so we only compare the family.
+        return CellUtil.matchingFamily(prevCell, currentCell) ? ReturnCode.NEXT_ROW : null;
     default:
-      throw new IllegalStateException("Received code is not valid.");
+        throw new IllegalStateException("Received code is not valid.");
     }
   }
 
@@ -240,7 +250,7 @@ public class FilterListWithOR extends FilterListBase {
   private void updatePrevCellList(int index, Cell currentCell, ReturnCode currentRC) {
     if (currentCell == null || currentRC == ReturnCode.INCLUDE || currentRC == ReturnCode.SKIP) {
       // If previous return code is INCLUDE or SKIP, we should always pass the next cell to the
-      // corresponding sub-filter(need not test shouldPassCurrentCellToFilter() method), So we
+      // corresponding sub-filter(need not test calculateReturnCodeByPrevCellAndRC() method), So we
       // need not save current cell to prevCellList for saving heap memory.
       prevCellList.set(index, null);
     } else {
@@ -254,28 +264,27 @@ public class FilterListWithOR extends FilterListBase {
       return ReturnCode.INCLUDE;
     }
     ReturnCode rc = null;
-    boolean everyFilterReturnHint = true;
     for (int i = 0, n = filters.size(); i < n; i++) {
       Filter filter = filters.get(i);
       subFiltersIncludedCell.set(i, false);
 
       Cell prevCell = this.prevCellList.get(i);
       ReturnCode prevCode = this.prevFilterRCList.get(i);
-      if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, prevCode)) {
-        everyFilterReturnHint = false;
+      if (filter.filterAllRemaining()) {
         continue;
       }
-
-      ReturnCode localRC = filter.filterCell(c);
+      ReturnCode localRC = calculateReturnCodeByPrevCellAndRC(filter, c, prevCell, prevCode);
+      if (localRC == null) {
+        // Can not get return code based on previous cell and previous return code. In other words,
+        // we should pass the current cell to this sub-filter to get the return code, and it won't
+        // impact the sub-filter's internal state.
+        localRC = filter.filterCell(c);
+      }
 
       // Update previous return code and previous cell for filter[i].
       updatePrevFilterRCList(i, localRC);
       updatePrevCellList(i, c, localRC);
 
-      if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) {
-        everyFilterReturnHint = false;
-      }
-
       rc = mergeReturnCode(rc, localRC);
 
       // For INCLUDE* case, we need to update the transformed cell.
@@ -284,15 +293,9 @@ public class FilterListWithOR extends FilterListBase {
         subFiltersIncludedCell.set(i, true);
       }
     }
-
-    if (everyFilterReturnHint) {
-      return ReturnCode.SEEK_NEXT_USING_HINT;
-    } else if (rc == null) {
-      // Each sub-filter in filter list got true for filterAllRemaining().
-      return ReturnCode.SKIP;
-    } else {
-      return rc;
-    }
+    // Each sub-filter in filter list got true for filterAllRemaining(), if rc is null, so we should
+    // return SKIP.
+    return rc == null ? ReturnCode.SKIP : rc;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/e160b5ac/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 91ca592..b318950 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
@@ -690,7 +690,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
         case SEEK_NEXT_USING_HINT:
           Cell nextKV = matcher.getNextKeyHint(cell);
-          if (nextKV != null) {
+          if (nextKV != null && comparator.compare(nextKV, cell) > 0) {
             seekAsDirection(nextKV);
             NextState stateAfterSeekByHint = needToReturn(outResult);
             if (stateAfterSeekByHint != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e160b5ac/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index 28f344a..2ba98ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -45,6 +45,7 @@ import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
@@ -604,15 +605,15 @@ public class TestFilterList {
         3, Bytes.toBytes("value"));
 
     assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterCell(kv1));
-    assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv2));
-    assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv3));
+    assertEquals(ReturnCode.NEXT_COL, filterList01.filterCell(kv2));
+    assertEquals(ReturnCode.NEXT_COL, filterList01.filterCell(kv3));
 
     FilterList filterList11 =
         new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1));
 
     assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv1));
-    assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv2));
-    assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv3));
+    assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv2));
+    assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv3));
   }
 
   @Test
@@ -631,9 +632,9 @@ public class TestFilterList {
         Bytes.toBytes("value"));
 
     assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
-    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv2));
+    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv2));
     assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3));
-    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv4));
+    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv4));
   }
 
   private static class MockFilter extends FilterBase {
@@ -1072,5 +1073,54 @@ public class TestFilterList {
     Assert.assertEquals(true, filter2.getTransformed());
     Assert.assertEquals(true, filter3.getTransformed());
   }
+
+  @Test
+  public void testFilterListWithORWhenPassingCellMismatchPreviousRC() throws IOException {
+    // Mainly test FilterListWithOR#calculateReturnCodeByPrevCellAndRC method with two sub-filters.
+    KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"),
+        100, Bytes.toBytes("value"));
+    KeyValue kv2 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 99,
+        Bytes.toBytes("value"));
+    KeyValue kv3 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 1,
+        Bytes.toBytes("value"));
+    KeyValue kv4 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fan"), Bytes.toBytes("a"), 1,
+        Bytes.toBytes("value"));
+    Filter subFilter1 = Mockito.mock(FilterBase.class);
+    Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
+    Mockito.when(subFilter1.filterCell(kv2)).thenReturn(ReturnCode.NEXT_COL);
+    Mockito.when(subFilter1.filterCell(kv3)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
+    Mockito.when(subFilter1.filterCell(kv4)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
+
+    Filter subFilter2 = Mockito.mock(FilterBase.class);
+    Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SKIP);
+    Mockito.when(subFilter2.filterCell(kv2)).thenReturn(ReturnCode.NEXT_ROW);
+    Mockito.when(subFilter2.filterCell(kv3)).thenReturn(ReturnCode.NEXT_ROW);
+    Mockito.when(subFilter2.filterCell(kv4)).thenReturn(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
+
+    Filter filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
+    Assert.assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
+    Assert.assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv2));
+    Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3));
+    Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv4));
+
+    // One sub-filter will filterAllRemaining but other sub-filter will return SEEK_HINT
+    subFilter1 = Mockito.mock(FilterBase.class);
+    Mockito.when(subFilter1.filterAllRemaining()).thenReturn(true);
+    Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.NEXT_ROW);
+
+    subFilter2 = Mockito.mock(FilterBase.class);
+    Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
+    filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
+    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
+
+    // Two sub-filter returns SEEK_NEXT_USING_HINT, then we should return SEEK_NEXT_USING_HINT.
+    subFilter1 = Mockito.mock(FilterBase.class);
+    Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
+
+    subFilter2 = Mockito.mock(FilterBase.class);
+    Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
+    filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
+    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e160b5ac/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
index 7967e0b..4266522 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -28,7 +29,6 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -90,4 +90,52 @@ public class TestFilterListOnMini {
       Assert.assertEquals(2, rr.size());
     }
   }
+
+  /**
+   * Test case for HBASE-21620
+   */
+  @Test
+  public void testColumnPrefixFilterConcatWithOR() throws Exception {
+    TableName tn = TableName.valueOf(name.getMethodName());
+    byte[] cf1 = Bytes.toBytes("f1");
+    byte[] row = Bytes.toBytes("row");
+    byte[] value = Bytes.toBytes("value");
+    String[] columns = new String[]{
+      "1544768273917010001_lt",
+      "1544768273917010001_w_1",
+      "1544768723910010001_ca_1",
+      "1544768723910010001_lt",
+      "1544768723910010001_ut_1",
+      "1544768723910010001_w_5",
+      "1544769779710010001_lt",
+      "1544769779710010001_w_5",
+      "1544769883529010001_lt",
+      "1544769883529010001_w_5",
+      "1544769915805010001_lt",
+      "1544769915805010001_w_5",
+      "1544779883529010001_lt",
+      "1544770422942010001_lt",
+      "1544770422942010001_w_5"
+    };
+    Table table = TEST_UTIL.createTable(tn, cf1);
+    for (int i = 0; i < columns.length; i++) {
+      Put put = new Put(row).addColumn(cf1, Bytes.toBytes(columns[i]), value);
+      table.put(put);
+    }
+    Scan scan = new Scan();
+    scan.withStartRow(row).withStopRow(row, true)
+        .setFilter(new FilterList(Operator.MUST_PASS_ONE,
+            new ColumnPrefixFilter(Bytes.toBytes("1544770422942010001_")),
+            new ColumnPrefixFilter(Bytes.toBytes("1544769883529010001_"))));
+    ResultScanner scanner = table.getScanner(scan);
+    Result result;
+    int resultCount = 0;
+    int cellCount = 0;
+    while ((result = scanner.next()) != null) {
+      cellCount += result.listCells().size();
+      resultCount++;
+    }
+    Assert.assertEquals(resultCount, 1);
+    Assert.assertEquals(cellCount, 4);
+  }
 }


[40/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
new file mode 100644
index 0000000..601d6b4
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
@@ -0,0 +1,69 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum TBloomFilterType implements org.apache.thrift.TEnum {
+  /**
+   * Bloomfilters disabled
+   */
+  NONE(0),
+  /**
+   * Bloom enabled with Table row as Key
+   */
+  ROW(1),
+  /**
+   * Bloom enabled with Table row &amp; column (family+qualifier) as Key
+   */
+  ROWCOL(2),
+  /**
+   * Bloom enabled with Table row prefix as Key, specify the length of the prefix
+   */
+  ROWPREFIX_FIXED_LENGTH(3),
+  /**
+   * Bloom enabled with Table row prefix as Key, specify the delimiter of the prefix
+   */
+  ROWPREFIX_DELIMITED(4);
+
+  private final int value;
+
+  private TBloomFilterType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TBloomFilterType findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return NONE;
+      case 1:
+        return ROW;
+      case 2:
+        return ROWCOL;
+      case 3:
+        return ROWPREFIX_FIXED_LENGTH;
+      case 4:
+        return ROWPREFIX_DELIMITED;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
index 7da4dda..464ac12 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
index d0d336c..24a7846 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * in a HBase table by column family and optionally
  * a column qualifier and timestamp
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 


[32/47] hbase git commit: HBASE-14939 Document bulk loaded hfile replication

Posted by zh...@apache.org.
HBASE-14939 Document bulk loaded hfile replication

Signed-off-by: Ashish Singhi <as...@apache.org>


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

Branch: refs/heads/HBASE-21512
Commit: c5520888779235a334583f7c369dcee49518e165
Parents: 4281cb3
Author: Wei-Chiu Chuang <we...@cloudera.com>
Authored: Wed Dec 26 20:14:18 2018 +0530
Committer: Ashish Singhi <as...@apache.org>
Committed: Wed Dec 26 20:14:18 2018 +0530

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/architecture.adoc | 32 ++++++++++++++++++----
 1 file changed, 26 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c5520888/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 17e9e13..27db26a 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -2543,12 +2543,6 @@ The most straightforward method is to either use the `TableOutputFormat` class f
 The bulk load feature uses a MapReduce job to output table data in HBase's internal data format, and then directly loads the generated StoreFiles into a running cluster.
 Using bulk load will use less CPU and network resources than simply using the HBase API.
 
-[[arch.bulk.load.limitations]]
-=== Bulk Load Limitations
-
-As bulk loading bypasses the write path, the WAL doesn't get written to as part of the process.
-Replication works by reading the WAL files so it won't see the bulk loaded data – and the same goes for the edits that use `Put.setDurability(SKIP_WAL)`. One way to handle that is to ship the raw files or the HFiles to the other cluster and do the other processing there.
-
 [[arch.bulk.load.arch]]
 === Bulk Load Architecture
 
@@ -2601,6 +2595,32 @@ To get started doing so, dig into `ImportTsv.java` and check the JavaDoc for HFi
 The import step of the bulk load can also be done programmatically.
 See the `LoadIncrementalHFiles` class for more information.
 
+[[arch.bulk.load.replication]]
+=== Bulk Loading Replication
+HBASE-13153 adds replication support for bulk loaded HFiles, available since HBase 1.3/2.0. This feature is enabled by setting `hbase.replication.bulkload.enabled` to `true` (default is `false`).
+You also need to copy the source cluster configuration files to the destination cluster.
+
+Additional configurations are required too:
+
+. `hbase.replication.source.fs.conf.provider`
++
+This defines the class which loads the source cluster file system client configuration in the destination cluster. This should be configured for all the RS in the destination cluster. Default is `org.apache.hadoop.hbase.replication.regionserver.DefaultSourceFSConfigurationProvider`.
++
+. `hbase.replication.conf.dir`
++
+This represents the base directory where the file system client configurations of the source cluster are copied to the destination cluster. This should be configured for all the RS in the destination cluster. Default is `$HBASE_CONF_DIR`.
++
+. `hbase.replication.cluster.id`
++
+This configuration is required in the cluster where replication for bulk loaded data is enabled. A source cluster is uniquely identified by the destination cluster using this id. This should be configured for all the RS in the source cluster configuration file for all the RS.
++
+
+
+
+For example: If source cluster FS client configurations are copied to the destination cluster under directory `/home/user/dc1/`, then `hbase.replication.cluster.id` should be configured as `dc1` and `hbase.replication.conf.dir` as `/home/user`.
+
+NOTE: `DefaultSourceFSConfigurationProvider` supports only `xml` type files. It loads source cluster FS client configuration only once, so if source cluster FS client configuration files are updated, every peer(s) cluster RS must be restarted to reload the configuration.
+
 [[arch.hdfs]]
 == HDFS
 


[33/47] hbase git commit: HBASE-21642 CopyTable by reading snapshot and bulkloading will save a lot of time

Posted by zh...@apache.org.
HBASE-21642 CopyTable by reading snapshot and bulkloading will save a lot of time


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

Branch: refs/heads/HBASE-21512
Commit: c2d5991b82e3b807cb11f5735ef5068b73720725
Parents: c552088
Author: huzheng <op...@gmail.com>
Authored: Wed Dec 26 16:17:55 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Thu Dec 27 18:22:54 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/CopyTable.java       | 109 ++++++++++++------
 .../hadoop/hbase/mapreduce/TestCopyTable.java   | 110 ++++++++++++++++---
 .../hbase/client/ClientSideRegionScanner.java   |  14 ++-
 .../hadoop/hbase/regionserver/HRegion.java      |   2 +-
 4 files changed, 187 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c2d5991b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
index 4e57f54..b59c9e6 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.mapreduce;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Random;
+import java.util.UUID;
 
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -29,6 +29,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
+import org.apache.hadoop.hbase.mapreduce.Import.Importer;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -70,8 +72,34 @@ public class CopyTable extends Configured implements Tool {
   boolean bulkload = false;
   Path bulkloadDir = null;
 
+  boolean readingSnapshot = false;
+  String snapshot = null;
+
   private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
 
+  private Path generateUniqTempDir(boolean withDirCreated) throws IOException {
+    FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
+    Path dir = new Path(fs.getWorkingDirectory(), NAME);
+    if (!fs.exists(dir)) {
+      fs.mkdirs(dir);
+    }
+    Path newDir = new Path(dir, UUID.randomUUID().toString());
+    if (withDirCreated) {
+      fs.mkdirs(newDir);
+    }
+    return newDir;
+  }
+
+  private void initCopyTableMapperReducerJob(Job job, Scan scan) throws IOException {
+    Class<? extends TableMapper> mapper = bulkload ? CellImporter.class : Importer.class;
+    if (readingSnapshot) {
+      TableMapReduceUtil.initTableSnapshotMapperJob(snapshot, scan, mapper, null, null, job, true,
+        generateUniqTempDir(true));
+    } else {
+      TableMapReduceUtil.initTableMapperJob(tableName, scan, mapper, null, null, job);
+    }
+  }
+
   /**
    * Sets up the actual job.
    *
@@ -79,13 +107,13 @@ public class CopyTable extends Configured implements Tool {
    * @return The newly created job.
    * @throws IOException When setting up the job fails.
    */
-  public Job createSubmittableJob(String[] args)
-  throws IOException {
+  public Job createSubmittableJob(String[] args) throws IOException {
     if (!doCommandLine(args)) {
       return null;
     }
 
-    Job job = Job.getInstance(getConf(), getConf().get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
+    String jobName = NAME + "_" + (tableName == null ? snapshot : tableName);
+    Job job = Job.getInstance(getConf(), getConf().get(JOB_NAME_CONF_KEY, jobName));
     job.setJarByClass(CopyTable.class);
     Scan scan = new Scan();
 
@@ -107,15 +135,15 @@ public class CopyTable extends Configured implements Tool {
       job.getConfiguration().set(TableInputFormat.SHUFFLE_MAPS, "true");
     }
     if (versions >= 0) {
-      scan.setMaxVersions(versions);
+      scan.readVersions(versions);
     }
 
     if (startRow != null) {
-      scan.setStartRow(Bytes.toBytesBinary(startRow));
+      scan.withStartRow(Bytes.toBytesBinary(startRow));
     }
 
     if (stopRow != null) {
-      scan.setStopRow(Bytes.toBytesBinary(stopRow));
+      scan.withStopRow(Bytes.toBytesBinary(stopRow));
     }
 
     if(families != null) {
@@ -140,24 +168,13 @@ public class CopyTable extends Configured implements Tool {
     job.setNumReduceTasks(0);
 
     if (bulkload) {
-      TableMapReduceUtil.initTableMapperJob(tableName, scan, Import.CellImporter.class, null, null,
-        job);
+      initCopyTableMapperReducerJob(job, scan);
 
       // We need to split the inputs by destination tables so that output of Map can be bulk-loaded.
       TableInputFormat.configureSplitTable(job, TableName.valueOf(dstTableName));
 
-      FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
-      Random rand = new Random();
-      Path root = new Path(fs.getWorkingDirectory(), "copytable");
-      fs.mkdirs(root);
-      while (true) {
-        bulkloadDir = new Path(root, "" + rand.nextLong());
-        if (!fs.exists(bulkloadDir)) {
-          break;
-        }
-      }
-
-      System.out.println("HFiles will be stored at " + this.bulkloadDir);
+      bulkloadDir = generateUniqTempDir(false);
+      LOG.info("HFiles will be stored at " + this.bulkloadDir);
       HFileOutputFormat2.setOutputPath(job, bulkloadDir);
       try (Connection conn = ConnectionFactory.createConnection(getConf());
           Admin admin = conn.getAdmin()) {
@@ -165,9 +182,7 @@ public class CopyTable extends Configured implements Tool {
           admin.getDescriptor((TableName.valueOf(dstTableName))));
       }
     } else {
-      TableMapReduceUtil.initTableMapperJob(tableName, scan,
-        Import.Importer.class, null, null, job);
-
+      initCopyTableMapperReducerJob(job, scan);
       TableMapReduceUtil.initTableReducerJob(dstTableName, null, job, null, peerAddress, null,
         null);
     }
@@ -183,7 +198,7 @@ public class CopyTable extends Configured implements Tool {
       System.err.println("ERROR: " + errorMsg);
     }
     System.err.println("Usage: CopyTable [general options] [--starttime=X] [--endtime=Y] " +
-        "[--new.name=NEW] [--peer.adr=ADR] <tablename>");
+        "[--new.name=NEW] [--peer.adr=ADR] <tablename | snapshotName>");
     System.err.println();
     System.err.println("Options:");
     System.err.println(" rs.class     hbase.regionserver.class of the peer cluster");
@@ -205,6 +220,7 @@ public class CopyTable extends Configured implements Tool {
     System.err.println(" all.cells    also copy delete markers and deleted cells");
     System.err.println(" bulkload     Write input into HFiles and bulk load to the destination "
         + "table");
+    System.err.println(" snapshot     Copy the data from snapshot to destination table.");
     System.err.println();
     System.err.println("Args:");
     System.err.println(" tablename    Name of the table to copy");
@@ -214,6 +230,12 @@ public class CopyTable extends Configured implements Tool {
     System.err.println(" $ hbase " +
         "org.apache.hadoop.hbase.mapreduce.CopyTable --starttime=1265875194289 --endtime=1265878794289 " +
         "--peer.adr=server1,server2,server3:2181:/hbase --families=myOldCf:myNewCf,cf2,cf3 TestTable ");
+    System.err.println(" To copy data from 'sourceTableSnapshot' to 'destTable': ");
+    System.err.println(" $ hbase org.apache.hadoop.hbase.mapreduce.CopyTable "
+        + "--snapshot --new.name=destTable sourceTableSnapshot");
+    System.err.println(" To copy data from 'sourceTableSnapshot' and bulk load to 'destTable': ");
+    System.err.println(" $ hbase org.apache.hadoop.hbase.mapreduce.CopyTable "
+        + "--new.name=destTable --snapshot --bulkload sourceTableSnapshot");
     System.err.println("For performance consider the following general option:\n"
         + "  It is recommended that you set the following to >=100. A higher value uses more memory but\n"
         + "  decreases the round trip time to the server and may increase performance.\n"
@@ -224,8 +246,6 @@ public class CopyTable extends Configured implements Tool {
   }
 
   private boolean doCommandLine(final String[] args) {
-    // Process command-line args. TODO: Better cmd-line processing
-    // (but hopefully something not as painful as cli options).
     if (args.length < 1) {
       printUsage(null);
       return false;
@@ -313,16 +333,24 @@ public class CopyTable extends Configured implements Tool {
           continue;
         }
 
-        if (i == args.length-1) {
-          tableName = cmd;
+        if(cmd.startsWith("--snapshot")){
+          readingSnapshot = true;
+          continue;
+        }
+
+        if (i == args.length - 1) {
+          if (readingSnapshot) {
+            snapshot = cmd;
+          } else {
+            tableName = cmd;
+          }
         } else {
           printUsage("Invalid argument '" + cmd + "'");
           return false;
         }
       }
       if (dstTableName == null && peerAddress == null) {
-        printUsage("At least a new table name or a " +
-            "peer address must be specified");
+        printUsage("At least a new table name or a peer address must be specified");
         return false;
       }
       if ((endTime != 0) && (startTime > endTime)) {
@@ -335,6 +363,22 @@ public class CopyTable extends Configured implements Tool {
         return false;
       }
 
+      if (readingSnapshot && peerAddress != null) {
+        printUsage("Loading data from snapshot to remote peer cluster is not supported.");
+        return false;
+      }
+
+      if (readingSnapshot && dstTableName == null) {
+        printUsage("The --new.name=<table> for destination table should be "
+            + "provided when copying data from snapshot .");
+        return false;
+      }
+
+      if (readingSnapshot && snapshot == null) {
+        printUsage("Snapshot shouldn't be null when --snapshot is enabled.");
+        return false;
+      }
+
       // set dstTableName if necessary
       if (dstTableName == null) {
         dstTableName = tableName;
@@ -371,6 +415,9 @@ public class CopyTable extends Configured implements Tool {
     }
     int code = 0;
     if (bulkload) {
+      LOG.info("Trying to bulk load data to destination table: " + dstTableName);
+      LOG.info("command: ./bin/hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles {} {}",
+        this.bulkloadDir.toString(), this.dstTableName);
       code = new LoadIncrementalHFiles(this.getConf())
           .run(new String[] { this.bulkloadDir.toString(), this.dstTableName });
       if (code == 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2d5991b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
index ed6857d..5591e5f 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
@@ -24,12 +24,15 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.io.PrintStream;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -45,6 +48,7 @@ import org.apache.hadoop.hbase.util.LauncherSecurityManager;
 import org.apache.hadoop.util.ToolRunner;
 
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -93,14 +97,9 @@ public class TestCopyTable {
     try (Table t1 = TEST_UTIL.createTable(tableName1, FAMILY);
          Table t2 = TEST_UTIL.createTable(tableName2, FAMILY)) {
       // put rows into the first table
-      for (int i = 0; i < 10; i++) {
-        Put p = new Put(Bytes.toBytes("row" + i));
-        p.addColumn(FAMILY, COLUMN1, COLUMN1);
-        t1.put(p);
-      }
+      loadData(t1, FAMILY, COLUMN1);
 
       CopyTable copy = new CopyTable();
-
       int code;
       if (bulkload) {
         code = ToolRunner.run(new Configuration(TEST_UTIL.getConfiguration()),
@@ -114,12 +113,7 @@ public class TestCopyTable {
       assertEquals("copy job failed", 0, code);
 
       // verify the data was copied into table 2
-      for (int i = 0; i < 10; i++) {
-        Get g = new Get(Bytes.toBytes("row" + i));
-        Result r = t2.get(g);
-        assertEquals(1, r.size());
-        assertTrue(CellUtil.matchingQualifier(r.rawCells()[0], COLUMN1));
-      }
+      verifyRows(t2, FAMILY, COLUMN1);
     } finally {
       TEST_UTIL.deleteTable(tableName1);
       TEST_UTIL.deleteTable(tableName2);
@@ -185,7 +179,6 @@ public class TestCopyTable {
               t2.getDescriptor().getValues().size());
       assertTrue("The mob row count is 0 but should be > 0",
               MobTestUtil.countMobRows(t2) > 0);
-
     } finally {
       TEST_UTIL.deleteTable(tableName1);
       TEST_UTIL.deleteTable(tableName2);
@@ -349,4 +342,95 @@ public class TestCopyTable {
         args);
     return status == 0;
   }
+
+  private void loadData(Table t, byte[] family, byte[] column) throws IOException {
+    for (int i = 0; i < 10; i++) {
+      byte[] row = Bytes.toBytes("row" + i);
+      Put p = new Put(row);
+      p.addColumn(family, column, row);
+      t.put(p);
+    }
+  }
+
+  private void verifyRows(Table t, byte[] family, byte[] column) throws IOException {
+    for (int i = 0; i < 10; i++) {
+      byte[] row = Bytes.toBytes("row" + i);
+      Get g = new Get(row).addFamily(family);
+      Result r = t.get(g);
+      Assert.assertNotNull(r);
+      Assert.assertEquals(1, r.size());
+      Cell cell = r.rawCells()[0];
+      Assert.assertTrue(CellUtil.matchingQualifier(cell, column));
+      Assert.assertEquals(Bytes.compareTo(cell.getValueArray(), cell.getValueOffset(),
+        cell.getValueLength(), row, 0, row.length), 0);
+    }
+  }
+
+  private Table createTable(TableName tableName, byte[] family, boolean isMob) throws IOException {
+    if (isMob) {
+      ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(family)
+          .setMobEnabled(true).setMobThreshold(1).build();
+      TableDescriptor desc =
+          TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(cfd).build();
+      return TEST_UTIL.createTable(desc, null);
+    } else {
+      return TEST_UTIL.createTable(tableName, family);
+    }
+  }
+
+  private void testCopyTableBySnapshot(String tablePrefix, boolean bulkLoad, boolean isMob)
+      throws Exception {
+    TableName table1 = TableName.valueOf(tablePrefix + 1);
+    TableName table2 = TableName.valueOf(tablePrefix + 2);
+    Table t1 = createTable(table1, FAMILY_A, isMob);
+    Table t2 = createTable(table2, FAMILY_A, isMob);
+    loadData(t1, FAMILY_A, Bytes.toBytes("qualifier"));
+    String snapshot = tablePrefix + "_snapshot";
+    TEST_UTIL.getAdmin().snapshot(snapshot, table1);
+    boolean success;
+    if (bulkLoad) {
+      success =
+          runCopy(new String[] { "--snapshot", "--new.name=" + table2, "--bulkload", snapshot });
+    } else {
+      success = runCopy(new String[] { "--snapshot", "--new.name=" + table2, snapshot });
+    }
+    Assert.assertTrue(success);
+    verifyRows(t2, FAMILY_A, Bytes.toBytes("qualifier"));
+  }
+
+  @Test
+  public void testLoadingSnapshotToTable() throws Exception {
+    testCopyTableBySnapshot("testLoadingSnapshotToTable", false, false);
+  }
+
+  @Test
+  public void tsetLoadingSnapshotToMobTable() throws Exception {
+    testCopyTableBySnapshot("testLoadingSnapshotToMobTable", false, true);
+  }
+
+  @Test
+  public void testLoadingSnapshotAndBulkLoadToTable() throws Exception {
+    testCopyTableBySnapshot("testLoadingSnapshotAndBulkLoadToTable", true, false);
+  }
+
+  @Test
+  public void testLoadingSnapshotAndBulkLoadToMobTable() throws Exception {
+    testCopyTableBySnapshot("testLoadingSnapshotAndBulkLoadToMobTable", true, true);
+  }
+
+  @Test
+  public void testLoadingSnapshotToRemoteCluster() throws Exception {
+    Assert.assertFalse(runCopy(
+      new String[] { "--snapshot", "--peerAdr=hbase://remoteHBase", "sourceSnapshotName" }));
+  }
+
+  @Test
+  public void testLoadingSnapshotWithoutSnapshotName() throws Exception {
+    Assert.assertFalse(runCopy(new String[] { "--snapshot", "--peerAdr=hbase://remoteHBase" }));
+  }
+
+  @Test
+  public void testLoadingSnapshotWithoutDestTable() throws Exception {
+    Assert.assertFalse(runCopy(new String[] { "--snapshot", "sourceSnapshotName" }));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2d5991b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
index 7a1a578..23a2399 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
@@ -26,11 +26,12 @@ 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.PrivateCellUtil;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,14 +51,21 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
 
   public ClientSideRegionScanner(Configuration conf, FileSystem fs,
       Path rootDir, TableDescriptor htd, RegionInfo hri, Scan scan, ScanMetrics scanMetrics)
-          throws IOException {
+      throws IOException {
     // region is immutable, set isolation level
     scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
 
     htd = TableDescriptorBuilder.newBuilder(htd).setReadOnly(true).build();
 
     // open region from the snapshot directory
-    this.region = HRegion.openHRegion(conf, fs, rootDir, hri, htd, null, null, null);
+    region = HRegion.newHRegion(FSUtils.getTableDir(rootDir, htd.getTableName()), null, fs, conf,
+      hri, htd, null);
+    // we won't initialize the MobFileCache when not running in RS process. so provided an
+    // initialized cache. Consider the case: an CF was set from an mob to non-mob. if we only
+    // initialize cache for MOB region, NPE from HMobStore will still happen. So Initialize the
+    // cache for every region although it may hasn't any mob CF, BTW the cache is very light-weight.
+    region.setMobFileCache(new MobFileCache(conf));
+    region.initialize();
 
     // create an internal region scanner
     this.scanner = region.getScanner(scan);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2d5991b/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 dc0fa22..9bf9309 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
@@ -7055,7 +7055,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param htd the table descriptor
    * @return the new instance
    */
-  static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
+  public static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
       Configuration conf, RegionInfo regionInfo, final TableDescriptor htd,
       RegionServerServices rsServices) {
     try {


[42/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2 (ADDENDUM add some comments)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
index 89a8a5e..8e53bdf 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
@@ -34,7 +34,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.client.TableDescriptor
+ */
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TTableDescriptor implements org.apache.thrift.TBase<TTableDescriptor, TTableDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TTableDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableDescriptor");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
index f2c0743..cec268a 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableName.java
@@ -34,7 +34,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.TableName
+ */
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TTableName implements org.apache.thrift.TBase<TTableName, TTableName._Fields>, java.io.Serializable, Cloneable, Comparable<TTableName> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableName");
 
@@ -47,12 +51,24 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
     schemes.put(TupleScheme.class, new TTableNameTupleSchemeFactory());
   }
 
+  /**
+   * namespace name
+   */
   public ByteBuffer ns; // required
+  /**
+   * tablename
+   */
   public ByteBuffer qualifier; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * namespace name
+     */
     NS((short)1, "ns"),
+    /**
+     * tablename
+     */
     QUALIFIER((short)2, "qualifier");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -157,6 +173,9 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
     this.qualifier = null;
   }
 
+  /**
+   * namespace name
+   */
   public byte[] getNs() {
     setNs(org.apache.thrift.TBaseHelper.rightSize(ns));
     return ns == null ? null : ns.array();
@@ -166,6 +185,9 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
     return org.apache.thrift.TBaseHelper.copyBinary(ns);
   }
 
+  /**
+   * namespace name
+   */
   public TTableName setNs(byte[] ns) {
     this.ns = ns == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(ns, ns.length));
     return this;
@@ -191,6 +213,9 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
     }
   }
 
+  /**
+   * tablename
+   */
   public byte[] getQualifier() {
     setQualifier(org.apache.thrift.TBaseHelper.rightSize(qualifier));
     return qualifier == null ? null : qualifier.array();
@@ -200,6 +225,9 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
     return org.apache.thrift.TBaseHelper.copyBinary(qualifier);
   }
 
+  /**
+   * tablename
+   */
   public TTableName setQualifier(byte[] qualifier) {
     this.qualifier = qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(qualifier, qualifier.length));
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
index 1e1898c..8ab746c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TTimeRange implements org.apache.thrift.TBase<TTimeRange, TTimeRange._Fields>, java.io.Serializable, Cloneable, Comparable<TTimeRange> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
index 6383329..c1b94ef 100644
--- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
+++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
@@ -315,6 +315,10 @@ enum TCompareOp {
   NO_OP = 6
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.regionserver.BloomType
+ */
 enum TBloomFilterType {
 /**
    * Bloomfilters disabled
@@ -338,6 +342,10 @@ enum TBloomFilterType {
   ROWPREFIX_DELIMITED = 4
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.io.compress.Algorithm
+ */
 enum TCompressionAlgorithm {
   LZO = 0,
   GZ = 1,
@@ -348,6 +356,10 @@ enum TCompressionAlgorithm {
   ZSTD = 6
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
+ */
 enum TDataBlockEncoding {
 /** Disable data block encoding. */
   NONE = 0,
@@ -361,6 +373,10 @@ enum TDataBlockEncoding {
   ROW_INDEX_V1 = 7
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.KeepDeletedCells
+ */
 enum TKeepDeletedCells {
   /** Deleted Cells are not retained. */
   FALSE = 0,
@@ -380,11 +396,21 @@ enum TKeepDeletedCells {
   TTL = 2
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.TableName
+ */
 struct TTableName {
+  /** namespace name */
   1: required binary ns
+  /** tablename */
   2: required binary qualifier
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
+ */
 struct TColumnFamilyDescriptor {
   1: required binary name
   2: optional map<binary, binary> attributes
@@ -409,6 +435,10 @@ struct TColumnFamilyDescriptor {
 
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.client.TableDescriptor
+ */
 struct TTableDescriptor {
  1: required TTableName tableName
  2: optional list<TColumnFamilyDescriptor> columns
@@ -416,6 +446,10 @@ struct TTableDescriptor {
  4: optional TDurability durability
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.NamespaceDescriptor
+ */
 struct TNamespaceDescriptor {
 1: required string name
 2: optional map<string, string> configuration
@@ -761,110 +795,238 @@ service THBaseService {
     7: required TRowMutations rowMutations
   ) throws (1: TIOError io)
 
+  /**
+  * Get a table descriptor.
+  * @return the TableDescriptor of the giving tablename
+  **/
   TTableDescriptor getTableDescriptor(
-  1: required TTableName table
+    /** the tablename of the table to get tableDescriptor*/
+    1: required TTableName table
   ) throws (1: TIOError io)
 
+  /**
+  * Get table descriptors of tables.
+  * @return the TableDescriptor of the giving tablename
+  **/
   list<TTableDescriptor> getTableDescriptors(
+    /** the tablename list of the tables to get tableDescriptor*/
     1: required list<TTableName> tables
   ) throws (1: TIOError io)
 
+  /**
+  *
+  * @return true if table exists already, false if not
+  **/
   bool tableExists(
-  1: TTableName tableName
+    /** the tablename of the tables to check*/
+    1: TTableName tableName
   ) throws (1: TIOError io)
 
+  /**
+  * Get table descriptors of tables that match the given pattern
+  * @return the tableDescriptors of the matching table
+  **/
   list<TTableDescriptor> getTableDescriptorsByPattern(
+    /** The regular expression to match against */
     1: required string regex
+    /** set to false if match only against userspace tables */
     2: required bool includeSysTables
   ) throws (1: TIOError io)
 
+  /**
+  * Get table descriptors of tables in the given namespace
+  * @return the tableDescriptors in the namespce
+  **/
   list<TTableDescriptor> getTableDescriptorsByNamespace(
-      1: required string name
+    /** The namesapce's name */
+    1: required string name
   ) throws (1: TIOError io)
 
+  /**
+  * Get table names of tables that match the given pattern
+  * @return the table names of the matching table
+  **/
   list<TTableName> getTableNamesByPattern(
+    /** The regular expression to match against */
     1: required string regex
+    /** set to false if match only against userspace tables */
     2: required bool includeSysTables
   ) throws (1: TIOError io)
 
+  /**
+  * Get table names of tables in the given namespace
+  * @return the table names of the matching table
+  **/
   list<TTableName> getTableNamesByNamespace(
+    /** The namesapce's name */
     1: required string name
   ) throws (1: TIOError io)
 
+  /**
+  * Creates a new table with an initial set of empty regions defined by the specified split keys.
+  * The total number of regions created will be the number of split keys plus one. Synchronous
+  * operation.
+  **/
   void createTable(
+    /** table descriptor for table */
     1: required TTableDescriptor desc
+    /** rray of split keys for the initial regions of the table */
     2: required list<binary> splitKeys
   ) throws (1: TIOError io)
 
+  /**
+  * Deletes a table. Synchronous operation.
+  **/
   void deleteTable(
+    /** the tablename to delete */
     1: required TTableName tableName
   ) throws (1: TIOError io)
 
+  /**
+  * Truncate a table. Synchronous operation.
+  **/
   void truncateTable(
+    /** the tablename to truncate */
     1: required TTableName tableName
+    /** whether to  preserve previous splits*/
     2: required bool preserveSplits
   ) throws (1: TIOError io)
 
+  /**
+  * Enalbe a table
+  **/
   void enableTable(
+    /** the tablename to enable */
     1: required TTableName tableName
   ) throws (1: TIOError io)
 
+  /**
+  * Disable a table
+  **/
   void disableTable(
+    /** the tablename to disable */
     1: required TTableName tableName
   ) throws (1: TIOError io)
 
+  /**
+  *
+  * @return true if table is enabled, false if not
+  **/
   bool isTableEnabled(
+    /** the tablename to check */
     1: required TTableName tableName
   ) throws (1: TIOError io)
 
+ /**
+  *
+  * @return true if table is disabled, false if not
+  **/
   bool isTableDisabled(
+    /** the tablename to check */
     1: required TTableName tableName
   ) throws (1: TIOError io)
 
+ /**
+  *
+  * @return true if table is available, false if not
+  **/
   bool isTableAvailable(
+    /** the tablename to check */
     1: required TTableName tableName
   ) throws (1: TIOError io)
 
+  /**
+   * Use this api to check if the table has been created with the specified number of splitkeys
+   * which was used while creating the given table. Note : If this api is used after a table's
+   * region gets splitted, the api may return false.
+   *
+   * @return true if table is available, false if not
+  **/
   bool isTableAvailableWithSplit(
+    /** the tablename to check */
     1: required TTableName tableName
+    /** keys to check if the table has been created with all split keys */
     2: required list<binary> splitKeys
   ) throws (1: TIOError io)
 
+  /**
+  * Add a column family to an existing table. Synchronous operation.
+  **/
   void addColumnFamily(
+    /** the tablename to add column family to */
     1: required TTableName tableName
+    /** column family descriptor of column family to be added */
     2: required TColumnFamilyDescriptor column
   ) throws (1: TIOError io)
 
+  /**
+  * Delete a column family from a table. Synchronous operation.
+  **/
   void deleteColumnFamily(
+    /** the tablename to delete column family from */
     1: required TTableName tableName
+    /** name of column family to be deleted */
     2: required binary column
   ) throws (1: TIOError io)
 
+  /**
+  * Modify an existing column family on a table. Synchronous operation.
+  **/
   void modifyColumnFamily(
+     /** the tablename to modify column family */
     1: required TTableName tableName
+    /** column family descriptor of column family to be modified */
     2: required TColumnFamilyDescriptor column
   ) throws (1: TIOError io)
 
+  /**
+  * Modify an existing table
+  **/
   void modifyTable(
+    /** the descriptor of the table to modify */
     1: required TTableDescriptor desc
   ) throws (1: TIOError io)
 
+  /**
+  * Create a new namespace. Blocks until namespace has been successfully created or an exception is
+  * thrown
+  **/
   void createNamespace(
+    /** descriptor which describes the new namespace */
     1: required TNamespaceDescriptor namespaceDesc
   ) throws (1: TIOError io)
 
+  /**
+  * Modify an existing namespace.  Blocks until namespace has been successfully modified or an
+  * exception is thrown
+  **/
   void modifyNamespace(
+    /** descriptor which describes the new namespace */
     1: required TNamespaceDescriptor namespaceDesc
   ) throws (1: TIOError io)
 
+  /**
+  * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+  * Blocks until namespace has been successfully deleted or an
+  * exception is thrown.
+  **/
   void deleteNamespace(
+    /** namespace name */
     1: required string name
   ) throws (1: TIOError io)
 
+  /**
+  *  Get a namespace descriptor by name.
+  *  @retrun the descriptor
+  **/
   TNamespaceDescriptor getNamespaceDescriptor(
+    /** name of namespace descriptor */
     1: required string name
   ) throws (1: TIOError io)
 
+  /**
+  * @return all namespaces
+  **/
   list<TNamespaceDescriptor> listNamespaceDescriptors(
   ) throws (1: TIOError io)
 }


[43/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2 (ADDENDUM add some comments)

Posted by zh...@apache.org.
HBASE-21650 Add DDL operation and some other miscellaneous to thrift2 (ADDENDUM add some comments)


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

Branch: refs/heads/HBASE-21512
Commit: b620334c20e84a4876226b508213ce11b8b187a0
Parents: 7820ba1
Author: Allan Yang <al...@apache.org>
Authored: Fri Dec 28 15:32:50 2018 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Fri Dec 28 15:32:50 2018 +0800

----------------------------------------------------------------------
 .../hbase/thrift/generated/AlreadyExists.java   |   2 +-
 .../hbase/thrift/generated/BatchMutation.java   |   2 +-
 .../thrift/generated/ColumnDescriptor.java      |   2 +-
 .../hadoop/hbase/thrift/generated/Hbase.java    |   2 +-
 .../hadoop/hbase/thrift/generated/IOError.java  |   2 +-
 .../hbase/thrift/generated/IllegalArgument.java |   2 +-
 .../hadoop/hbase/thrift/generated/Mutation.java |   2 +-
 .../hadoop/hbase/thrift/generated/TAppend.java  |   2 +-
 .../hadoop/hbase/thrift/generated/TCell.java    |   2 +-
 .../hadoop/hbase/thrift/generated/TColumn.java  |   2 +-
 .../hbase/thrift/generated/TIncrement.java      |   2 +-
 .../hbase/thrift/generated/TRegionInfo.java     |   2 +-
 .../hbase/thrift/generated/TRowResult.java      |   2 +-
 .../hadoop/hbase/thrift/generated/TScan.java    |   2 +-
 .../hadoop/hbase/thrift2/generated/TAppend.java |   2 +-
 .../hbase/thrift2/generated/TAuthorization.java |   2 +-
 .../thrift2/generated/TBloomFilterType.java     |   4 +
 .../thrift2/generated/TCellVisibility.java      |   2 +-
 .../hadoop/hbase/thrift2/generated/TColumn.java |   2 +-
 .../generated/TColumnFamilyDescriptor.java      |   6 +-
 .../thrift2/generated/TColumnIncrement.java     |   2 +-
 .../hbase/thrift2/generated/TColumnValue.java   |   2 +-
 .../generated/TCompressionAlgorithm.java        |   4 +
 .../thrift2/generated/TDataBlockEncoding.java   |   4 +
 .../hadoop/hbase/thrift2/generated/TDelete.java |   2 +-
 .../hadoop/hbase/thrift2/generated/TGet.java    |   2 +-
 .../hbase/thrift2/generated/THBaseService.java  | 571 ++++++++++++++++++-
 .../hbase/thrift2/generated/THRegionInfo.java   |   2 +-
 .../thrift2/generated/THRegionLocation.java     |   2 +-
 .../hbase/thrift2/generated/TIOError.java       |   2 +-
 .../thrift2/generated/TIllegalArgument.java     |   2 +-
 .../hbase/thrift2/generated/TIncrement.java     |   2 +-
 .../thrift2/generated/TKeepDeletedCells.java    |   4 +
 .../thrift2/generated/TNamespaceDescriptor.java |   6 +-
 .../hadoop/hbase/thrift2/generated/TPut.java    |   2 +-
 .../hadoop/hbase/thrift2/generated/TResult.java |   2 +-
 .../hbase/thrift2/generated/TRowMutations.java  |   2 +-
 .../hadoop/hbase/thrift2/generated/TScan.java   |   2 +-
 .../hbase/thrift2/generated/TServerName.java    |   2 +-
 .../thrift2/generated/TTableDescriptor.java     |   6 +-
 .../hbase/thrift2/generated/TTableName.java     |  30 +-
 .../hbase/thrift2/generated/TTimeRange.java     |   2 +-
 .../apache/hadoop/hbase/thrift2/hbase.thrift    | 168 +++++-
 43 files changed, 828 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
index 8ec3e32..4457b9f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class AlreadyExists extends TException implements org.apache.thrift.TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
index 0872223..f605286 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A BatchMutation object is used to apply a number of Mutations to a single row.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class BatchMutation implements org.apache.thrift.TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
index bccd48b..c0e885d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * such as the number of versions, compression settings, etc. It is
  * used as input when creating a table or adding a column.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class ColumnDescriptor implements org.apache.thrift.TBase<ColumnDescriptor, ColumnDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
index dc4d8bc..54bb4ce 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class Hbase {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
index a32b008..09d076c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * to the Hbase master or an Hbase region server.  Also used to return
  * more general Hbase error conditions.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class IOError extends TException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable, Comparable<IOError> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
index 573c496..bfe4380 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * An IllegalArgument exception indicates an illegal or invalid
  * argument was passed into a procedure.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class IllegalArgument extends TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
index 72925b7..967e3d1 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A Mutation object is used to either update or delete a column-value.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class Mutation implements org.apache.thrift.TBase<Mutation, Mutation._Fields>, java.io.Serializable, Cloneable, Comparable<Mutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
index a21b5f7..599f645 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * An Append object is used to specify the parameters for performing the append operation.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
index afe0af1..c8f3576 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
  * the timestamp of a cell to a first-class value, making it easy to take
  * note of temporal data. Cell is used all the way from HStore up to HTable.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TCell implements org.apache.thrift.TBase<TCell, TCell._Fields>, java.io.Serializable, Cloneable, Comparable<TCell> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
index 0115bbd..0ce1d75 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Holds column name and the cell.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
index 7c018f1..6c0827b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * For increments that are not incrementColumnValue
  * equivalents.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
index ec75247..d2d3e0e 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A TRegionInfo contains information about an HTable region.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TRegionInfo implements org.apache.thrift.TBase<TRegionInfo, TRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TRegionInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
index b09a368..6520007 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Holds row name and then a map of columns to cells.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResult._Fields>, java.io.Serializable, Cloneable, Comparable<TRowResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
index 97d59c7..fcc237d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A Scan object is used to specify scanner parameters when opening a scanner.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
index c89f67b..1b9d6fc 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
index 1b168d8..916dc0c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
index 601d6b4..c4022a0 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
@@ -11,6 +11,10 @@ import java.util.Map;
 import java.util.HashMap;
 import org.apache.thrift.TEnum;
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.regionserver.BloomType
+ */
 public enum TBloomFilterType implements org.apache.thrift.TEnum {
   /**
    * Bloomfilters disabled

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
index 464ac12..e9af9fe 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
index 24a7846..11e3ace 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * in a HBase table by column family and optionally
  * a column qualifier and timestamp
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
index 03cb2f6..df4656f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
@@ -34,7 +34,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
+ */
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TColumnFamilyDescriptor implements org.apache.thrift.TBase<TColumnFamilyDescriptor, TColumnFamilyDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnFamilyDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
index 0f27519..18c5ff2 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Represents a single cell and the amount to increment it by
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
index 6cded1b..1355141 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Represents a single cell and its value.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
index 46799be..59f7448 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompressionAlgorithm.java
@@ -11,6 +11,10 @@ import java.util.Map;
 import java.util.HashMap;
 import org.apache.thrift.TEnum;
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.io.compress.Algorithm
+ */
 public enum TCompressionAlgorithm implements org.apache.thrift.TEnum {
   LZO(0),
   GZ(1),

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
index 1594aac3..bd513e9 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDataBlockEncoding.java
@@ -11,6 +11,10 @@ import java.util.Map;
 import java.util.HashMap;
 import org.apache.thrift.TEnum;
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
+ */
 public enum TDataBlockEncoding implements org.apache.thrift.TEnum {
   /**
    * Disable data block encoding.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
index fb4ffe6..cdce4fb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
@@ -60,7 +60,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields>, java.io.Serializable, Cloneable, Comparable<TDelete> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
index 1b01ef5..b38d936 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
  * If you specify a time range and a timestamp the range is ignored.
  * Timestamps on TColumns are ignored.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.io.Serializable, Cloneable, Comparable<TGet> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
index 7388443..bb5db4d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class THBaseService {
 
   public interface Iface {
@@ -282,54 +282,239 @@ public class THBaseService {
      */
     public boolean checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Get a table descriptor.
+     * @return the TableDescriptor of the giving tablename
+     * 
+     * 
+     * @param table the tablename of the table to get tableDescriptor
+     */
     public TTableDescriptor getTableDescriptor(TTableName table) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Get table descriptors of tables.
+     * @return the TableDescriptor of the giving tablename
+     * 
+     * 
+     * @param tables the tablename list of the tables to get tableDescriptor
+     */
     public List<TTableDescriptor> getTableDescriptors(List<TTableName> tables) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * 
+     * @return true if table exists already, false if not
+     * 
+     * 
+     * @param tableName the tablename of the tables to check
+     */
     public boolean tableExists(TTableName tableName) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Get table descriptors of tables that match the given pattern
+     * @return the tableDescriptors of the matching table
+     * 
+     * 
+     * @param regex The regular expression to match against
+     * 
+     * @param includeSysTables set to false if match only against userspace tables
+     */
     public List<TTableDescriptor> getTableDescriptorsByPattern(String regex, boolean includeSysTables) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Get table descriptors of tables in the given namespace
+     * @return the tableDescriptors in the namespce
+     * 
+     * 
+     * @param name The namesapce's name
+     */
     public List<TTableDescriptor> getTableDescriptorsByNamespace(String name) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Get table names of tables that match the given pattern
+     * @return the table names of the matching table
+     * 
+     * 
+     * @param regex The regular expression to match against
+     * 
+     * @param includeSysTables set to false if match only against userspace tables
+     */
     public List<TTableName> getTableNamesByPattern(String regex, boolean includeSysTables) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Get table names of tables in the given namespace
+     * @return the table names of the matching table
+     * 
+     * 
+     * @param name The namesapce's name
+     */
     public List<TTableName> getTableNamesByNamespace(String name) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Creates a new table with an initial set of empty regions defined by the specified split keys.
+     * The total number of regions created will be the number of split keys plus one. Synchronous
+     * operation.
+     * 
+     * 
+     * @param desc table descriptor for table
+     * 
+     * @param splitKeys rray of split keys for the initial regions of the table
+     */
     public void createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Deletes a table. Synchronous operation.
+     * 
+     * 
+     * @param tableName the tablename to delete
+     */
     public void deleteTable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Truncate a table. Synchronous operation.
+     * 
+     * 
+     * @param tableName the tablename to truncate
+     * 
+     * @param preserveSplits whether to  preserve previous splits
+     */
     public void truncateTable(TTableName tableName, boolean preserveSplits) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Enalbe a table
+     * 
+     * 
+     * @param tableName the tablename to enable
+     */
     public void enableTable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Disable a table
+     * 
+     * 
+     * @param tableName the tablename to disable
+     */
     public void disableTable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * 
+     * @return true if table is enabled, false if not
+     * 
+     * 
+     * @param tableName the tablename to check
+     */
     public boolean isTableEnabled(TTableName tableName) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * 
+     * @return true if table is disabled, false if not
+     * 
+     * 
+     * @param tableName the tablename to check
+     */
     public boolean isTableDisabled(TTableName tableName) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * 
+     * @return true if table is available, false if not
+     * 
+     * 
+     * @param tableName the tablename to check
+     */
     public boolean isTableAvailable(TTableName tableName) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     *  * Use this api to check if the table has been created with the specified number of splitkeys
+     *  * which was used while creating the given table. Note : If this api is used after a table's
+     *  * region gets splitted, the api may return false.
+     *  *
+     *  * @return true if table is available, false if not
+     * *
+     * 
+     * @param tableName the tablename to check
+     * 
+     * @param splitKeys keys to check if the table has been created with all split keys
+     */
     public boolean isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Add a column family to an existing table. Synchronous operation.
+     * 
+     * 
+     * @param tableName the tablename to add column family to
+     * 
+     * @param column column family descriptor of column family to be added
+     */
     public void addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Delete a column family from a table. Synchronous operation.
+     * 
+     * 
+     * @param tableName the tablename to delete column family from
+     * 
+     * @param column name of column family to be deleted
+     */
     public void deleteColumnFamily(TTableName tableName, ByteBuffer column) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Modify an existing column family on a table. Synchronous operation.
+     * 
+     * 
+     * @param tableName the tablename to modify column family
+     * 
+     * @param column column family descriptor of column family to be modified
+     */
     public void modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Modify an existing table
+     * 
+     * 
+     * @param desc the descriptor of the table to modify
+     */
     public void modifyTable(TTableDescriptor desc) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Create a new namespace. Blocks until namespace has been successfully created or an exception is
+     * thrown
+     * 
+     * 
+     * @param namespaceDesc descriptor which describes the new namespace
+     */
     public void createNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Modify an existing namespace.  Blocks until namespace has been successfully modified or an
+     * exception is thrown
+     * 
+     * 
+     * @param namespaceDesc descriptor which describes the new namespace
+     */
     public void modifyNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+     * Blocks until namespace has been successfully deleted or an
+     * exception is thrown.
+     * 
+     * 
+     * @param name namespace name
+     */
     public void deleteNamespace(String name) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Get a namespace descriptor by name.
+     * @retrun the descriptor
+     * 
+     * 
+     * @param name name of namespace descriptor
+     */
     public TNamespaceDescriptor getNamespaceDescriptor(String name) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * @return all namespaces
+     * 
+     */
     public List<TNamespaceDescriptor> listNamespaceDescriptors() throws TIOError, org.apache.thrift.TException;
 
   }
@@ -28057,10 +28242,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new getTableDescriptor_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename of the table to get tableDescriptor
+     */
     public TTableName table; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename of the table to get tableDescriptor
+       */
       TABLE((short)1, "table");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -28155,10 +28346,16 @@ public class THBaseService {
       this.table = null;
     }
 
+    /**
+     * the tablename of the table to get tableDescriptor
+     */
     public TTableName getTable() {
       return this.table;
     }
 
+    /**
+     * the tablename of the table to get tableDescriptor
+     */
     public getTableDescriptor_args setTable(TTableName table) {
       this.table = table;
       return this;
@@ -28889,10 +29086,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new getTableDescriptors_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename list of the tables to get tableDescriptor
+     */
     public List<TTableName> tables; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename list of the tables to get tableDescriptor
+       */
       TABLES((short)1, "tables");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -29007,10 +29210,16 @@ public class THBaseService {
       this.tables.add(elem);
     }
 
+    /**
+     * the tablename list of the tables to get tableDescriptor
+     */
     public List<TTableName> getTables() {
       return this.tables;
     }
 
+    /**
+     * the tablename list of the tables to get tableDescriptor
+     */
     public getTableDescriptors_args setTables(List<TTableName> tables) {
       this.tables = tables;
       return this;
@@ -29819,10 +30028,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new tableExists_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename of the tables to check
+     */
     public TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename of the tables to check
+       */
       TABLE_NAME((short)1, "tableName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -29917,10 +30132,16 @@ public class THBaseService {
       this.tableName = null;
     }
 
+    /**
+     * the tablename of the tables to check
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename of the tables to check
+     */
     public tableExists_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -30654,12 +30875,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new getTableDescriptorsByPattern_argsTupleSchemeFactory());
     }
 
+    /**
+     * The regular expression to match against
+     */
     public String regex; // required
+    /**
+     * set to false if match only against userspace tables
+     */
     public boolean includeSysTables; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * The regular expression to match against
+       */
       REGEX((short)1, "regex"),
+      /**
+       * set to false if match only against userspace tables
+       */
       INCLUDE_SYS_TABLES((short)2, "includeSysTables");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -30767,10 +31000,16 @@ public class THBaseService {
       this.includeSysTables = false;
     }
 
+    /**
+     * The regular expression to match against
+     */
     public String getRegex() {
       return this.regex;
     }
 
+    /**
+     * The regular expression to match against
+     */
     public getTableDescriptorsByPattern_args setRegex(String regex) {
       this.regex = regex;
       return this;
@@ -30791,10 +31030,16 @@ public class THBaseService {
       }
     }
 
+    /**
+     * set to false if match only against userspace tables
+     */
     public boolean isIncludeSysTables() {
       return this.includeSysTables;
     }
 
+    /**
+     * set to false if match only against userspace tables
+     */
     public getTableDescriptorsByPattern_args setIncludeSysTables(boolean includeSysTables) {
       this.includeSysTables = includeSysTables;
       setIncludeSysTablesIsSet(true);
@@ -31629,10 +31874,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new getTableDescriptorsByNamespace_argsTupleSchemeFactory());
     }
 
+    /**
+     * The namesapce's name
+     */
     public String name; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * The namesapce's name
+       */
       NAME((short)1, "name");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -31727,10 +31978,16 @@ public class THBaseService {
       this.name = null;
     }
 
+    /**
+     * The namesapce's name
+     */
     public String getName() {
       return this.name;
     }
 
+    /**
+     * The namesapce's name
+     */
     public getTableDescriptorsByNamespace_args setName(String name) {
       this.name = name;
       return this;
@@ -32506,12 +32763,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new getTableNamesByPattern_argsTupleSchemeFactory());
     }
 
+    /**
+     * The regular expression to match against
+     */
     public String regex; // required
+    /**
+     * set to false if match only against userspace tables
+     */
     public boolean includeSysTables; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * The regular expression to match against
+       */
       REGEX((short)1, "regex"),
+      /**
+       * set to false if match only against userspace tables
+       */
       INCLUDE_SYS_TABLES((short)2, "includeSysTables");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -32619,10 +32888,16 @@ public class THBaseService {
       this.includeSysTables = false;
     }
 
+    /**
+     * The regular expression to match against
+     */
     public String getRegex() {
       return this.regex;
     }
 
+    /**
+     * The regular expression to match against
+     */
     public getTableNamesByPattern_args setRegex(String regex) {
       this.regex = regex;
       return this;
@@ -32643,10 +32918,16 @@ public class THBaseService {
       }
     }
 
+    /**
+     * set to false if match only against userspace tables
+     */
     public boolean isIncludeSysTables() {
       return this.includeSysTables;
     }
 
+    /**
+     * set to false if match only against userspace tables
+     */
     public getTableNamesByPattern_args setIncludeSysTables(boolean includeSysTables) {
       this.includeSysTables = includeSysTables;
       setIncludeSysTablesIsSet(true);
@@ -33481,10 +33762,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new getTableNamesByNamespace_argsTupleSchemeFactory());
     }
 
+    /**
+     * The namesapce's name
+     */
     public String name; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * The namesapce's name
+       */
       NAME((short)1, "name");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -33579,10 +33866,16 @@ public class THBaseService {
       this.name = null;
     }
 
+    /**
+     * The namesapce's name
+     */
     public String getName() {
       return this.name;
     }
 
+    /**
+     * The namesapce's name
+     */
     public getTableNamesByNamespace_args setName(String name) {
       this.name = name;
       return this;
@@ -34358,12 +34651,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new createTable_argsTupleSchemeFactory());
     }
 
+    /**
+     * table descriptor for table
+     */
     public TTableDescriptor desc; // required
+    /**
+     * rray of split keys for the initial regions of the table
+     */
     public List<ByteBuffer> splitKeys; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * table descriptor for table
+       */
       DESC((short)1, "desc"),
+      /**
+       * rray of split keys for the initial regions of the table
+       */
       SPLIT_KEYS((short)2, "splitKeys");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -34470,10 +34775,16 @@ public class THBaseService {
       this.splitKeys = null;
     }
 
+    /**
+     * table descriptor for table
+     */
     public TTableDescriptor getDesc() {
       return this.desc;
     }
 
+    /**
+     * table descriptor for table
+     */
     public createTable_args setDesc(TTableDescriptor desc) {
       this.desc = desc;
       return this;
@@ -34509,10 +34820,16 @@ public class THBaseService {
       this.splitKeys.add(elem);
     }
 
+    /**
+     * rray of split keys for the initial regions of the table
+     */
     public List<ByteBuffer> getSplitKeys() {
       return this.splitKeys;
     }
 
+    /**
+     * rray of split keys for the initial regions of the table
+     */
     public createTable_args setSplitKeys(List<ByteBuffer> splitKeys) {
       this.splitKeys = splitKeys;
       return this;
@@ -35229,10 +35546,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new deleteTable_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to delete
+     */
     public TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to delete
+       */
       TABLE_NAME((short)1, "tableName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -35327,10 +35650,16 @@ public class THBaseService {
       this.tableName = null;
     }
 
+    /**
+     * the tablename to delete
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to delete
+     */
     public deleteTable_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -35952,12 +36281,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new truncateTable_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to truncate
+     */
     public TTableName tableName; // required
+    /**
+     * whether to  preserve previous splits
+     */
     public boolean preserveSplits; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to truncate
+       */
       TABLE_NAME((short)1, "tableName"),
+      /**
+       * whether to  preserve previous splits
+       */
       PRESERVE_SPLITS((short)2, "preserveSplits");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -36065,10 +36406,16 @@ public class THBaseService {
       this.preserveSplits = false;
     }
 
+    /**
+     * the tablename to truncate
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to truncate
+     */
     public truncateTable_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -36089,10 +36436,16 @@ public class THBaseService {
       }
     }
 
+    /**
+     * whether to  preserve previous splits
+     */
     public boolean isPreserveSplits() {
       return this.preserveSplits;
     }
 
+    /**
+     * whether to  preserve previous splits
+     */
     public truncateTable_args setPreserveSplits(boolean preserveSplits) {
       this.preserveSplits = preserveSplits;
       setPreserveSplitsIsSet(true);
@@ -36773,10 +37126,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new enableTable_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to enable
+     */
     public TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to enable
+       */
       TABLE_NAME((short)1, "tableName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -36871,10 +37230,16 @@ public class THBaseService {
       this.tableName = null;
     }
 
+    /**
+     * the tablename to enable
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to enable
+     */
     public enableTable_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -37495,10 +37860,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new disableTable_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to disable
+     */
     public TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to disable
+       */
       TABLE_NAME((short)1, "tableName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -37593,10 +37964,16 @@ public class THBaseService {
       this.tableName = null;
     }
 
+    /**
+     * the tablename to disable
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to disable
+     */
     public disableTable_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -38217,10 +38594,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new isTableEnabled_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to check
+       */
       TABLE_NAME((short)1, "tableName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -38315,10 +38698,16 @@ public class THBaseService {
       this.tableName = null;
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to check
+     */
     public isTableEnabled_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -39044,10 +39433,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new isTableDisabled_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to check
+       */
       TABLE_NAME((short)1, "tableName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -39142,10 +39537,16 @@ public class THBaseService {
       this.tableName = null;
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to check
+     */
     public isTableDisabled_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -39871,10 +40272,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new isTableAvailable_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName tableName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to check
+       */
       TABLE_NAME((short)1, "tableName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -39969,10 +40376,16 @@ public class THBaseService {
       this.tableName = null;
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to check
+     */
     public isTableAvailable_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -40699,12 +41112,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new isTableAvailableWithSplit_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName tableName; // required
+    /**
+     * keys to check if the table has been created with all split keys
+     */
     public List<ByteBuffer> splitKeys; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to check
+       */
       TABLE_NAME((short)1, "tableName"),
+      /**
+       * keys to check if the table has been created with all split keys
+       */
       SPLIT_KEYS((short)2, "splitKeys");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -40811,10 +41236,16 @@ public class THBaseService {
       this.splitKeys = null;
     }
 
+    /**
+     * the tablename to check
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to check
+     */
     public isTableAvailableWithSplit_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -40850,10 +41281,16 @@ public class THBaseService {
       this.splitKeys.add(elem);
     }
 
+    /**
+     * keys to check if the table has been created with all split keys
+     */
     public List<ByteBuffer> getSplitKeys() {
       return this.splitKeys;
     }
 
+    /**
+     * keys to check if the table has been created with all split keys
+     */
     public isTableAvailableWithSplit_args setSplitKeys(List<ByteBuffer> splitKeys) {
       this.splitKeys = splitKeys;
       return this;
@@ -41676,12 +42113,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new addColumnFamily_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to add column family to
+     */
     public TTableName tableName; // required
+    /**
+     * column family descriptor of column family to be added
+     */
     public TColumnFamilyDescriptor column; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to add column family to
+       */
       TABLE_NAME((short)1, "tableName"),
+      /**
+       * column family descriptor of column family to be added
+       */
       COLUMN((short)2, "column");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -41786,10 +42235,16 @@ public class THBaseService {
       this.column = null;
     }
 
+    /**
+     * the tablename to add column family to
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to add column family to
+     */
     public addColumnFamily_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -41810,10 +42265,16 @@ public class THBaseService {
       }
     }
 
+    /**
+     * column family descriptor of column family to be added
+     */
     public TColumnFamilyDescriptor getColumn() {
       return this.column;
     }
 
+    /**
+     * column family descriptor of column family to be added
+     */
     public addColumnFamily_args setColumn(TColumnFamilyDescriptor column) {
       this.column = column;
       return this;
@@ -42504,12 +42965,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new deleteColumnFamily_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to delete column family from
+     */
     public TTableName tableName; // required
+    /**
+     * name of column family to be deleted
+     */
     public ByteBuffer column; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to delete column family from
+       */
       TABLE_NAME((short)1, "tableName"),
+      /**
+       * name of column family to be deleted
+       */
       COLUMN((short)2, "column");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -42614,10 +43087,16 @@ public class THBaseService {
       this.column = null;
     }
 
+    /**
+     * the tablename to delete column family from
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to delete column family from
+     */
     public deleteColumnFamily_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -42638,6 +43117,9 @@ public class THBaseService {
       }
     }
 
+    /**
+     * name of column family to be deleted
+     */
     public byte[] getColumn() {
       setColumn(org.apache.thrift.TBaseHelper.rightSize(column));
       return column == null ? null : column.array();
@@ -42647,6 +43129,9 @@ public class THBaseService {
       return org.apache.thrift.TBaseHelper.copyBinary(column);
     }
 
+    /**
+     * name of column family to be deleted
+     */
     public deleteColumnFamily_args setColumn(byte[] column) {
       this.column = column == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(column, column.length));
       return this;
@@ -43337,12 +43822,24 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new modifyColumnFamily_argsTupleSchemeFactory());
     }
 
+    /**
+     * the tablename to modify column family
+     */
     public TTableName tableName; // required
+    /**
+     * column family descriptor of column family to be modified
+     */
     public TColumnFamilyDescriptor column; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the tablename to modify column family
+       */
       TABLE_NAME((short)1, "tableName"),
+      /**
+       * column family descriptor of column family to be modified
+       */
       COLUMN((short)2, "column");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -43447,10 +43944,16 @@ public class THBaseService {
       this.column = null;
     }
 
+    /**
+     * the tablename to modify column family
+     */
     public TTableName getTableName() {
       return this.tableName;
     }
 
+    /**
+     * the tablename to modify column family
+     */
     public modifyColumnFamily_args setTableName(TTableName tableName) {
       this.tableName = tableName;
       return this;
@@ -43471,10 +43974,16 @@ public class THBaseService {
       }
     }
 
+    /**
+     * column family descriptor of column family to be modified
+     */
     public TColumnFamilyDescriptor getColumn() {
       return this.column;
     }
 
+    /**
+     * column family descriptor of column family to be modified
+     */
     public modifyColumnFamily_args setColumn(TColumnFamilyDescriptor column) {
       this.column = column;
       return this;
@@ -44164,10 +44673,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new modifyTable_argsTupleSchemeFactory());
     }
 
+    /**
+     * the descriptor of the table to modify
+     */
     public TTableDescriptor desc; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * the descriptor of the table to modify
+       */
       DESC((short)1, "desc");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -44262,10 +44777,16 @@ public class THBaseService {
       this.desc = null;
     }
 
+    /**
+     * the descriptor of the table to modify
+     */
     public TTableDescriptor getDesc() {
       return this.desc;
     }
 
+    /**
+     * the descriptor of the table to modify
+     */
     public modifyTable_args setDesc(TTableDescriptor desc) {
       this.desc = desc;
       return this;
@@ -44886,10 +45407,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new createNamespace_argsTupleSchemeFactory());
     }
 
+    /**
+     * descriptor which describes the new namespace
+     */
     public TNamespaceDescriptor namespaceDesc; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * descriptor which describes the new namespace
+       */
       NAMESPACE_DESC((short)1, "namespaceDesc");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -44984,10 +45511,16 @@ public class THBaseService {
       this.namespaceDesc = null;
     }
 
+    /**
+     * descriptor which describes the new namespace
+     */
     public TNamespaceDescriptor getNamespaceDesc() {
       return this.namespaceDesc;
     }
 
+    /**
+     * descriptor which describes the new namespace
+     */
     public createNamespace_args setNamespaceDesc(TNamespaceDescriptor namespaceDesc) {
       this.namespaceDesc = namespaceDesc;
       return this;
@@ -45608,10 +46141,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new modifyNamespace_argsTupleSchemeFactory());
     }
 
+    /**
+     * descriptor which describes the new namespace
+     */
     public TNamespaceDescriptor namespaceDesc; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * descriptor which describes the new namespace
+       */
       NAMESPACE_DESC((short)1, "namespaceDesc");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -45706,10 +46245,16 @@ public class THBaseService {
       this.namespaceDesc = null;
     }
 
+    /**
+     * descriptor which describes the new namespace
+     */
     public TNamespaceDescriptor getNamespaceDesc() {
       return this.namespaceDesc;
     }
 
+    /**
+     * descriptor which describes the new namespace
+     */
     public modifyNamespace_args setNamespaceDesc(TNamespaceDescriptor namespaceDesc) {
       this.namespaceDesc = namespaceDesc;
       return this;
@@ -46330,10 +46875,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new deleteNamespace_argsTupleSchemeFactory());
     }
 
+    /**
+     * namespace name
+     */
     public String name; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * namespace name
+       */
       NAME((short)1, "name");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -46428,10 +46979,16 @@ public class THBaseService {
       this.name = null;
     }
 
+    /**
+     * namespace name
+     */
     public String getName() {
       return this.name;
     }
 
+    /**
+     * namespace name
+     */
     public deleteNamespace_args setName(String name) {
       this.name = name;
       return this;
@@ -47047,10 +47604,16 @@ public class THBaseService {
       schemes.put(TupleScheme.class, new getNamespaceDescriptor_argsTupleSchemeFactory());
     }
 
+    /**
+     * name of namespace descriptor
+     */
     public String name; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * name of namespace descriptor
+       */
       NAME((short)1, "name");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -47145,10 +47708,16 @@ public class THBaseService {
       this.name = null;
     }
 
+    /**
+     * name of namespace descriptor
+     */
     public String getName() {
       return this.name;
     }
 
+    /**
+     * name of namespace descriptor
+     */
     public getNamespaceDescriptor_args setName(String name) {
       this.name = name;
       return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
index 8450f5b..b430d4d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class THRegionInfo implements org.apache.thrift.TBase<THRegionInfo, THRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionInfo");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
index b1146e9..f64951e 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class THRegionLocation implements org.apache.thrift.TBase<THRegionLocation, THRegionLocation._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionLocation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
index 9569c3f..ccb8514 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * to the HBase master or a HBase region server. Also used to return
  * more general HBase error conditions.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TIOError extends TException implements org.apache.thrift.TBase<TIOError, TIOError._Fields>, java.io.Serializable, Cloneable, Comparable<TIOError> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
index 6734dec..4436fc0 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * A TIllegalArgument exception indicates an illegal or invalid
  * argument was passed into a procedure.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TIllegalArgument extends TException implements org.apache.thrift.TBase<TIllegalArgument, TIllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<TIllegalArgument> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
index 3663e61..70d8eab 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
index 4ce64e2..24f4d09 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
@@ -11,6 +11,10 @@ import java.util.Map;
 import java.util.HashMap;
 import org.apache.thrift.TEnum;
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.KeepDeletedCells
+ */
 public enum TKeepDeletedCells implements org.apache.thrift.TEnum {
   /**
    * Deleted Cells are not retained.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
index 72d4916..1c0b41b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
@@ -34,7 +34,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.NamespaceDescriptor
+ */
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TNamespaceDescriptor implements org.apache.thrift.TBase<TNamespaceDescriptor, TNamespaceDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TNamespaceDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TNamespaceDescriptor");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
index ad82c67..5739d78 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
@@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.io.Serializable, Cloneable, Comparable<TPut> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
index c19cf89..3d40ea2 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * if no Result is found, row and columnValues will not be set.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields>, java.io.Serializable, Cloneable, Comparable<TResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
index a3ab58e..cc846f5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A TRowMutations object is used to apply a number of Mutations to a single row.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRowMutations._Fields>, java.io.Serializable, Cloneable, Comparable<TRowMutations> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
index ddcedb4..99046cc 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * Any timestamps in the columns are ignored but the colFamTimeRangeMap included, use timeRange to select by timestamp.
  * Max versions defaults to 1.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b620334c/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
index 65f963d..e7e0372 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
 public class TServerName implements org.apache.thrift.TBase<TServerName, TServerName._Fields>, java.io.Serializable, Cloneable, Comparable<TServerName> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName");
 


[30/47] hbase git commit: HBASE-21631 (addendum) Fixed TestQuotasShell failure (quotas_test.rb)

Posted by zh...@apache.org.
HBASE-21631 (addendum) Fixed TestQuotasShell failure (quotas_test.rb)

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

Branch: refs/heads/HBASE-21512
Commit: 44dec60054d1c45880d591c74a023f7a534e6d73
Parents: dbafa1b
Author: Sakthi <sa...@gmail.com>
Authored: Sun Dec 23 21:25:07 2018 -0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Dec 24 14:15:59 2018 +0800

----------------------------------------------------------------------
 hbase-shell/src/test/ruby/hbase/quotas_test.rb | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/44dec600/hbase-shell/src/test/ruby/hbase/quotas_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/quotas_test.rb b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
index be6b238..295d545 100644
--- a/hbase-shell/src/test/ruby/hbase/quotas_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/quotas_test.rb
@@ -99,8 +99,7 @@ module Hbase
     define_test 'can set and remove quota' do
       command(:set_quota, TYPE => SPACE, LIMIT => '1G', POLICY => NO_INSERTS, TABLE => @test_name)
       output = capture_stdout{ command(:list_quotas) }
-      size = 1024 * 1024 * 1024
-      assert(output.include?("LIMIT => #{size}"))
+      assert(output.include?("LIMIT => 1G"))
       assert(output.include?("VIOLATION_POLICY => NO_INSERTS"))
       assert(output.include?("TYPE => SPACE"))
       assert(output.include?("TABLE => #{@test_name}"))


[14/47] hbase git commit: HBASE-21514 Refactor CacheConfig

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
index 444102d..2065c0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
@@ -28,12 +28,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -104,17 +106,15 @@ public class TestScannerSelectionUsingTTL {
   @Test
   public void testScannerSelection() throws IOException {
     Configuration conf = TEST_UTIL.getConfiguration();
-    CacheConfig.instantiateBlockCache(conf);
     conf.setBoolean("hbase.store.delete.expired.storefile", false);
-    HColumnDescriptor hcd =
-      new HColumnDescriptor(FAMILY_BYTES)
-          .setMaxVersions(Integer.MAX_VALUE)
-          .setTimeToLive(TTL_SECONDS);
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
-    htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(TABLE);
-    HRegion region = HBaseTestingUtility.createRegionAndWAL(info,
-      TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd);
+    LruBlockCache cache = (LruBlockCache) BlockCacheFactory.createBlockCache(conf);
+
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE).setColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_BYTES).setMaxVersions(Integer.MAX_VALUE)
+            .setTimeToLive(TTL_SECONDS).build()).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(TABLE).build();
+    HRegion region = HBaseTestingUtility
+        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, td, cache);
 
     long ts = EnvironmentEdgeManager.currentTime();
     long version = 0; //make sure each new set of Put's have a new ts
@@ -136,10 +136,7 @@ public class TestScannerSelectionUsingTTL {
       version++;
     }
 
-    Scan scan = new Scan();
-    scan.setMaxVersions(Integer.MAX_VALUE);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    LruBlockCache cache = (LruBlockCache) cacheConf.getBlockCache();
+    Scan scan = new Scan().readVersions(Integer.MAX_VALUE);
     cache.clearCache();
     InternalScanner scanner = region.getScanner(scan);
     List<Cell> results = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 844b705..a930d7f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Random;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentSkipListMap;
@@ -48,8 +49,10 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionSizeStore;
@@ -708,4 +711,14 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   public TableDescriptors getTableDescriptors() {
     return null;
   }
+
+  @Override
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.empty();
+  }
+
+  @Override
+  public Optional<MobFileCache> getMobFileCache() {
+    return Optional.empty();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
index 75d9019..7ef8eca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
@@ -79,7 +79,13 @@ public class TestMasterNotCarryTable {
   @Test
   public void testMasterBlockCache() {
     // no need to instantiate block cache.
-    assertNull(master.getCacheConfig().getBlockCache());
+    assertFalse(master.getBlockCache().isPresent());
+  }
+
+  @Test
+  public void testMasterMOBFileCache() {
+    // no need to instantiate mob file cache.
+    assertFalse(master.getMobFileCache().isPresent());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
index 325a580..b5bbd53 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.regionserver.HMobStore;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@@ -57,7 +60,6 @@ public class TestMobFileCache {
   private HBaseTestingUtility UTIL;
   private HRegion region;
   private Configuration conf;
-  private MobCacheConfig mobCacheConf;
   private MobFileCache mobFileCache;
   private Date currentDate = new Date();
   private static final String TEST_CACHE_SIZE = "2";
@@ -85,6 +87,7 @@ public class TestMobFileCache {
   public void setUp() throws Exception {
     UTIL = HBaseTestingUtility.createLocalHTU();
     conf = UTIL.getConfiguration();
+    conf.set(MobConstants.MOB_FILE_CACHE_SIZE_KEY, TEST_CACHE_SIZE);
     HTableDescriptor htd = UTIL.createTableDescriptor("testMobFileCache");
     HColumnDescriptor hcd1 = new HColumnDescriptor(FAMILY1);
     hcd1.setMobEnabled(true);
@@ -98,7 +101,10 @@ public class TestMobFileCache {
     htd.addFamily(hcd1);
     htd.addFamily(hcd2);
     htd.addFamily(hcd3);
-    region = UTIL.createLocalHRegion(htd, null, null);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    mobFileCache = new MobFileCache(conf);
+    region = HBaseTestingUtility
+        .createRegionAndWAL(regionInfo, UTIL.getDataTestDir(), conf, htd, mobFileCache);
   }
 
   @After
@@ -121,7 +127,6 @@ public class TestMobFileCache {
     HColumnDescriptor hcd = new HColumnDescriptor(family);
     hcd.setMaxVersions(4);
     hcd.setMobEnabled(true);
-    mobCacheConf = new MobCacheConfig(conf, hcd);
     return createMobStoreFile(hcd);
   }
 
@@ -158,17 +163,16 @@ public class TestMobFileCache {
   @Test
   public void testMobFileCache() throws Exception {
     FileSystem fs = FileSystem.get(conf);
-    conf.set(MobConstants.MOB_FILE_CACHE_SIZE_KEY, TEST_CACHE_SIZE);
-    mobFileCache = new MobFileCache(conf);
     Path file1Path = createMobStoreFile(FAMILY1);
     Path file2Path = createMobStoreFile(FAMILY2);
     Path file3Path = createMobStoreFile(FAMILY3);
 
+    CacheConfig cacheConf = new CacheConfig(conf);
     // Before open one file by the MobFileCache
     assertEquals(EXPECTED_CACHE_SIZE_ZERO, mobFileCache.getCacheSize());
     // Open one file by the MobFileCache
     CachedMobFile cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
-        fs, file1Path, mobCacheConf);
+        fs, file1Path, cacheConf);
     assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
     assertNotNull(cachedMobFile1);
     assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
@@ -188,13 +192,13 @@ public class TestMobFileCache {
 
     // Reopen three cached file
     cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
-        fs, file1Path, mobCacheConf);
+        fs, file1Path, cacheConf);
     assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
     CachedMobFile cachedMobFile2 = (CachedMobFile) mobFileCache.openFile(
-        fs, file2Path, mobCacheConf);
+        fs, file2Path, cacheConf);
     assertEquals(EXPECTED_CACHE_SIZE_TWO, mobFileCache.getCacheSize());
     CachedMobFile cachedMobFile3 = (CachedMobFile) mobFileCache.openFile(
-        fs, file3Path, mobCacheConf);
+        fs, file3Path, cacheConf);
     // Before the evict
     // Evict the cache, should close the first file 1
     assertEquals(EXPECTED_CACHE_SIZE_THREE, mobFileCache.getCacheSize());

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 59f6678..0fb183e 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
@@ -740,12 +740,7 @@ public class DataBlockEncodingTool {
         "Excluding the first " + benchmarkNOmit + " times from statistics.");
 
     final Configuration conf = HBaseConfiguration.create();
-    try {
-      testCodecs(conf, kvLimit, pathName, compressionName, doBenchmark,
-          doVerify);
-    } finally {
-      (new CacheConfig(conf)).getBlockCache().shutdown();
-    }
+    testCodecs(conf, kvLimit, pathName, compressionName, doBenchmark, doVerify);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 57948b6..dcb6c3f 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
@@ -180,6 +180,6 @@ public class EncodedSeekPerformanceTest {
   }
 
   private void clearBlockCache() {
-    ((LruBlockCache) cacheConf.getBlockCache()).clearCache();
+    ((LruBlockCache) cacheConf.getBlockCache().get()).clearCache();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index 16381d2..b449f37 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -114,14 +115,17 @@ public class TestAtomicOperation {
   @After
   public void teardown() throws IOException {
     if (region != null) {
-      BlockCache bc = region.getStores().get(0).getCacheConfig().getBlockCache();
+      CacheConfig cacheConfig = region.getStores().get(0).getCacheConfig();
       region.close();
       WAL wal = region.getWAL();
-      if (wal != null) wal.close();
-      if (bc != null) bc.shutdown();
+      if (wal != null) {
+        wal.close();
+      }
+      cacheConfig.getBlockCache().ifPresent(BlockCache::shutdown);
       region = null;
     }
   }
+
   //////////////////////////////////////////////////////////////////////////////
   // New tests that doesn't spin up a mini cluster but rather just test the
   // individual code pieces in the HRegion.

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
index 2cf3f8c..4425762 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
@@ -30,24 +30,28 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 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.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 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.EnvironmentEdgeManagerTestHelper;
-import org.junit.*;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
@@ -61,12 +65,12 @@ public class TestBlocksRead  {
       HBaseClassTestRule.forClass(TestBlocksRead.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestBlocksRead.class);
-  @Rule public TestName testName = new TestName();
+  @Rule
+  public TestName testName = new TestName();
 
   static final BloomType[] BLOOM_TYPE = new BloomType[] { BloomType.ROWCOL,
       BloomType.ROW, BloomType.NONE };
 
-  private static BlockCache blockCache;
   HRegion region = null;
   private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final String DIR = TEST_UTIL.getDataTestDir("TestBlocksRead").toString();
@@ -76,7 +80,6 @@ public class TestBlocksRead  {
   public static void setUp() throws Exception {
     // disable compactions in this test.
     TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10000);
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
   }
 
   @AfterClass
@@ -93,23 +96,31 @@ public class TestBlocksRead  {
    * @throws IOException
    * @return created and initialized region.
    */
-  private HRegion initHRegion(byte[] tableName, String callingMethod,
-      Configuration conf, String family) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
-    HColumnDescriptor familyDesc;
+  private HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
+      String family) throws IOException {
+    return initHRegion(tableName, callingMethod, conf, family, null);
+  }
+
+  /**
+   * Callers must afterward call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
+   */
+  private HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
+      String family, BlockCache blockCache) throws IOException {
+    TableDescriptorBuilder builder =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
     for (int i = 0; i < BLOOM_TYPE.length; i++) {
       BloomType bloomType = BLOOM_TYPE[i];
-      familyDesc = new HColumnDescriptor(family + "_" + bloomType)
-          .setBlocksize(1)
-          .setBloomFilterType(BLOOM_TYPE[i]);
-      htd.addFamily(familyDesc);
+      builder.setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family + "_" + bloomType))
+              .setBlocksize(1).setBloomFilterType(bloomType).build());
     }
-
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build();
     Path path = new Path(DIR + callingMethod);
-    HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
-    blockCache = new CacheConfig(conf).getBlockCache();
-    return r;
+    if (blockCache != null) {
+      return HBaseTestingUtility.createRegionAndWAL(info, path, conf, builder.build(), blockCache);
+    } else {
+      return HBaseTestingUtility.createRegionAndWAL(info, path, conf, builder.build());
+    }
   }
 
   private void putData(String family, String row, String col, long version)
@@ -204,10 +215,6 @@ public class TestBlocksRead  {
       return HFile.DATABLOCK_READ_COUNT.sum();
   }
 
-  private static long getBlkCount() {
-    return blockCache.getBlockCount();
-  }
-
   /**
    * Test # of blocks read for some simple seek cases.
    *
@@ -381,7 +388,8 @@ public class TestBlocksRead  {
     byte [] TABLE = Bytes.toBytes("testBlocksReadWhenCachingDisabled");
     String FAMILY = "cf1";
 
-    this.region = initHRegion(TABLE, testName.getMethodName(), conf, FAMILY);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(conf);
+    this.region = initHRegion(TABLE, testName.getMethodName(), conf, FAMILY, blockCache);
 
     try {
       putData(FAMILY, "row", "col1", 1);
@@ -390,7 +398,7 @@ public class TestBlocksRead  {
 
       // Execute a scan with caching turned off
       // Expected blocks stored: 0
-      long blocksStart = getBlkCount();
+      long blocksStart = blockCache.getBlockCount();
       Scan scan = new Scan();
       scan.setCacheBlocks(false);
       RegionScanner rs = region.getScanner(scan);
@@ -398,7 +406,7 @@ public class TestBlocksRead  {
       rs.next(result);
       assertEquals(2 * BLOOM_TYPE.length, result.size());
       rs.close();
-      long blocksEnd = getBlkCount();
+      long blocksEnd = blockCache.getBlockCount();
 
       assertEquals(blocksStart, blocksEnd);
 
@@ -411,7 +419,7 @@ public class TestBlocksRead  {
       rs.next(result);
       assertEquals(2 * BLOOM_TYPE.length, result.size());
       rs.close();
-      blocksEnd = getBlkCount();
+      blocksEnd = blockCache.getBlockCount();
 
       assertEquals(2 * BLOOM_TYPE.length, blocksEnd - blocksStart);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
index 0ba4e97..c7bc168 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
@@ -17,25 +17,33 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.HBaseTestCase.addContent;
+import static org.junit.Assert.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -43,7 +51,7 @@ import org.junit.experimental.categories.Category;
 
 @SuppressWarnings("deprecation")
 @Category({RegionServerTests.class, SmallTests.class})
-public class TestBlocksScanned extends HBaseTestCase {
+public class TestBlocksScanned {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -56,13 +64,14 @@ public class TestBlocksScanned extends HBaseTestCase {
   private static int BLOCK_SIZE = 70;
 
   private static HBaseTestingUtility TEST_UTIL = null;
+  private Configuration conf;
+  private Path testDir;
 
-  @Override
   @Before
   public void setUp() throws Exception {
-    super.setUp();
     TEST_UTIL = new HBaseTestingUtility();
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
+    conf = TEST_UTIL.getConfiguration();
+    testDir = TEST_UTIL.getDataTestDir("TestBlocksScanned");
   }
 
   @Test
@@ -96,12 +105,16 @@ public class TestBlocksScanned extends HBaseTestCase {
     _testBlocksScanned(table);
   }
 
-  private void _testBlocksScanned(HTableDescriptor table) throws Exception {
-    HRegion r = createNewHRegion(table, START_KEY, END_KEY, TEST_UTIL.getConfiguration());
+  private void _testBlocksScanned(TableDescriptor td) throws Exception {
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(conf);
+    RegionInfo regionInfo =
+        RegionInfoBuilder.newBuilder(td.getTableName()).setStartKey(START_KEY).setEndKey(END_KEY)
+            .build();
+    HRegion r = HBaseTestingUtility.createRegionAndWAL(regionInfo, testDir, conf, td, blockCache);
     addContent(r, FAMILY, COL);
     r.flush(true);
 
-    CacheStats stats = new CacheConfig(TEST_UTIL.getConfiguration()).getBlockCache().getStats();
+    CacheStats stats = blockCache.getStats();
     long before = stats.getHitCount() + stats.getMissCount();
     // Do simple test of getting one row only first.
     Scan scan = new Scan().withStartRow(Bytes.toBytes("aaa")).withStopRow(Bytes.toBytes("aaz"))
@@ -119,11 +132,12 @@ public class TestBlocksScanned extends HBaseTestCase {
 
     int kvPerBlock = (int) Math.ceil(BLOCK_SIZE /
         (double) KeyValueUtil.ensureKeyValue(results.get(0)).getLength());
-    Assert.assertEquals(2, kvPerBlock);
+    assertEquals(2, kvPerBlock);
 
     long expectDataBlockRead = (long) Math.ceil(expectResultSize / (double) kvPerBlock);
     long expectIndexBlockRead = expectDataBlockRead;
 
-    assertEquals(expectIndexBlockRead+expectDataBlockRead, stats.getHitCount() + stats.getMissCount() - before);
+    assertEquals(expectIndexBlockRead + expectDataBlockRead,
+        stats.getHitCount() + stats.getMissCount() - before);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 dc51dae..1248b21 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -161,8 +162,6 @@ public class TestCacheOnWriteInSchema {
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
     conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, false);
     conf.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, false);
-    CacheConfig.instantiateBlockCache(conf);
-
     fs = HFileSystem.get(conf);
 
     // Create the schema
@@ -182,6 +181,7 @@ public class TestCacheOnWriteInSchema {
     walFactory = new WALFactory(conf, id);
 
     region = TEST_UTIL.createLocalHRegion(info, htd, walFactory.getWAL(info));
+    region.setBlockCache(BlockCacheFactory.createBlockCache(conf));
     store = new HStore(region, hcd, conf);
   }
 
@@ -224,7 +224,7 @@ public class TestCacheOnWriteInSchema {
 
   private void readStoreFile(Path path) throws IOException {
     CacheConfig cacheConf = store.getCacheConfig();
-    BlockCache cache = cacheConf.getBlockCache();
+    BlockCache cache = cacheConf.getBlockCache().get();
     HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
     sf.initReader();
     HFile.Reader reader = sf.getReader().getHFileReader();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
index 47fc2d7..3c8e8be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
@@ -100,8 +100,8 @@ public class TestClearRegionBlockCache {
 
   @Test
   public void testClearBlockCache() throws Exception {
-    BlockCache blockCache1 = rs1.getCacheConfig().getBlockCache();
-    BlockCache blockCache2 = rs2.getCacheConfig().getBlockCache();
+    BlockCache blockCache1 = rs1.getBlockCache().get();
+    BlockCache blockCache2 = rs2.getBlockCache().get();
 
     long initialBlockCount1 = blockCache1.getBlockCount();
     long initialBlockCount2 = blockCache2.getBlockCount();
@@ -125,23 +125,24 @@ public class TestClearRegionBlockCache {
   public void testClearBlockCacheFromAdmin() throws Exception {
     Admin admin = HTU.getAdmin();
 
-    // All RS run in a same process, so the block cache is same for rs1 and rs2
-    BlockCache blockCache = rs1.getCacheConfig().getBlockCache();
-    long initialBlockCount = blockCache.getBlockCount();
+    BlockCache blockCache1 = rs1.getBlockCache().get();
+    BlockCache blockCache2 = rs2.getBlockCache().get();
+    long initialBlockCount1 = blockCache1.getBlockCount();
+    long initialBlockCount2 = blockCache2.getBlockCount();
 
     // scan will cause blocks to be added in BlockCache
     scanAllRegionsForRS(rs1);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
+    assertEquals(blockCache1.getBlockCount() - initialBlockCount1,
+        HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
     scanAllRegionsForRS(rs2);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
-          + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
+    assertEquals(blockCache2.getBlockCount() - initialBlockCount2,
+        HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
 
     CacheEvictionStats stats = admin.clearBlockCache(TABLE_NAME);
     assertEquals(stats.getEvictedBlocks(), HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
         + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
-    assertEquals(initialBlockCount, blockCache.getBlockCount());
+    assertEquals(initialBlockCount1, blockCache1.getBlockCount());
+    assertEquals(initialBlockCount2, blockCache2.getBlockCount());
   }
 
   @Test
@@ -149,23 +150,24 @@ public class TestClearRegionBlockCache {
     AsyncAdmin admin =
         ConnectionFactory.createAsyncConnection(HTU.getConfiguration()).get().getAdmin();
 
-    // All RS run in a same process, so the block cache is same for rs1 and rs2
-    BlockCache blockCache = rs1.getCacheConfig().getBlockCache();
-    long initialBlockCount = blockCache.getBlockCount();
+    BlockCache blockCache1 = rs1.getBlockCache().get();
+    BlockCache blockCache2 = rs2.getBlockCache().get();
+    long initialBlockCount1 = blockCache1.getBlockCount();
+    long initialBlockCount2 = blockCache2.getBlockCount();
 
     // scan will cause blocks to be added in BlockCache
     scanAllRegionsForRS(rs1);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
+    assertEquals(blockCache1.getBlockCount() - initialBlockCount1,
+        HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
     scanAllRegionsForRS(rs2);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
-          + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
+    assertEquals(blockCache2.getBlockCount() - initialBlockCount2,
+        HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
 
     CacheEvictionStats stats = admin.clearBlockCache(TABLE_NAME).get();
-    assertEquals(stats.getEvictedBlocks(), HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
-        + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
-    assertEquals(initialBlockCount, blockCache.getBlockCount());
+    assertEquals(stats.getEvictedBlocks(), HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY) + HTU
+        .getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
+    assertEquals(initialBlockCount1, blockCache1.getBlockCount());
+    assertEquals(initialBlockCount2, blockCache2.getBlockCount());
   }
 
   private void scanAllRegionsForRS(HRegionServer rs) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 4c2d645..3d22b73 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
@@ -41,6 +40,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
 import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
@@ -122,7 +122,6 @@ public class TestCompoundBloomFilter {
   private static Configuration conf;
   private static CacheConfig cacheConf;
   private FileSystem fs;
-  private BlockCache blockCache;
 
   /** A message of the form "in test#&lt;number>:" to include in logging. */
   private String testIdMsg;
@@ -130,6 +129,8 @@ public class TestCompoundBloomFilter {
   private static final int GENERATION_SEED = 2319;
   private static final int EVALUATION_SEED = 135;
 
+  private BlockCache blockCache;
+
   @Before
   public void setUp() throws IOException {
     conf = TEST_UTIL.getConfiguration();
@@ -138,11 +139,8 @@ public class TestCompoundBloomFilter {
     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
 
     fs = FileSystem.get(conf);
-
-    CacheConfig.instantiateBlockCache(conf);
-    cacheConf = new CacheConfig(conf);
-    blockCache = cacheConf.getBlockCache();
-    assertNotNull(blockCache);
+    blockCache = BlockCacheFactory.createBlockCache(conf);
+    cacheConf = new CacheConfig(conf, blockCache);
   }
 
   private List<KeyValue> createSortedKeyValues(Random rand, int n) {
@@ -305,7 +303,7 @@ public class TestCompoundBloomFilter {
     conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
         BLOOM_BLOCK_SIZES[t]);
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, blockCache);
     HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCK_SIZES[t]).build();
     StoreFileWriter w = new StoreFileWriter.Builder(conf, cacheConf, fs)
             .withOutputDir(TEST_UTIL.getDataTestDir())
@@ -373,7 +371,5 @@ public class TestCompoundBloomFilter {
       rowColKV.getRowLength()));
     assertEquals(0, rowKV.getQualifierLength());
   }
-
-
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 9c5a667..2001602 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
@@ -32,6 +32,7 @@ import java.util.NavigableSet;
 import java.util.Optional;
 import java.util.concurrent.ConcurrentSkipListSet;
 import javax.crypto.spec.SecretKeySpec;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -42,20 +43,24 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.io.crypto.aes.AES;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -90,7 +95,6 @@ public class TestHMobStore {
 
   private HMobStore store;
   private HRegion region;
-  private HColumnDescriptor hcd;
   private FileSystem fs;
   private byte [] table = Bytes.toBytes("table");
   private byte [] family = Bytes.toBytes("family");
@@ -131,49 +135,44 @@ public class TestHMobStore {
       byte [] next = iter.next();
       expected.add(new KeyValue(row, family, next, 1, value));
       get.addColumn(family, next);
-      get.setMaxVersions(); // all versions.
+      get.readAllVersions();
     }
   }
 
-  private void init(String methodName, Configuration conf, boolean testStore)
-  throws IOException {
-    hcd = new HColumnDescriptor(family);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(3L);
-    hcd.setMaxVersions(4);
-    init(methodName, conf, hcd, testStore);
+  private void init(String methodName, Configuration conf, boolean testStore) throws IOException {
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true).setMobThreshold(3L)
+            .setMaxVersions(4).build();
+    init(methodName, conf, cfd, testStore);
   }
 
-  private void init(String methodName, Configuration conf,
-      HColumnDescriptor hcd, boolean testStore) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
-    init(methodName, conf, htd, hcd, testStore);
-  }
+  private void init(String methodName, Configuration conf, ColumnFamilyDescriptor cfd,
+      boolean testStore) throws IOException {
+    TableDescriptor td =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(table)).setColumnFamily(cfd).build();
 
-  private void init(String methodName, Configuration conf, HTableDescriptor htd,
-      HColumnDescriptor hcd, boolean testStore) throws IOException {
     //Setting up tje Region and Store
-    Path basedir = new Path(DIR+methodName);
-    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
+    Path basedir = new Path(DIR + methodName);
+    Path tableDir = FSUtils.getTableDir(basedir, td.getTableName());
     String logName = "logs";
     Path logdir = new Path(basedir, logName);
     FileSystem fs = FileSystem.get(conf);
     fs.delete(logdir, true);
 
-    htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    RegionInfo info = RegionInfoBuilder.newBuilder(td.getTableName()).build();
     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, methodName);
-    region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, htd, null);
-    store = new HMobStore(region, hcd, conf);
-    if(testStore) {
-      init(conf, hcd);
+    region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, td, null);
+    region.setMobFileCache(new MobFileCache(conf));
+    store = new HMobStore(region, cfd, conf);
+    if (testStore) {
+      init(conf, cfd);
     }
   }
 
-  private void init(Configuration conf, HColumnDescriptor hcd)
+  private void init(Configuration conf, ColumnFamilyDescriptor cfd)
       throws IOException {
     Path basedir = FSUtils.getRootDir(conf);
     fs = FileSystem.get(conf);
@@ -187,7 +186,7 @@ public class TestHMobStore {
     KeyValue[] keys = new KeyValue[] { key1, key2, key3 };
     int maxKeyCount = keys.length;
     StoreFileWriter mobWriter = store.createWriterInTmp(currentDate, maxKeyCount,
-        hcd.getCompactionCompressionType(), region.getRegionInfo().getStartKey(), false);
+        cfd.getCompactionCompressionType(), region.getRegionInfo().getStartKey(), false);
     mobFilePath = mobWriter.getPath();
 
     mobWriter.append(key1);
@@ -382,15 +381,11 @@ public class TestHMobStore {
    */
   @Test
   public void testMobCellSizeThreshold() throws IOException {
-
     final Configuration conf = HBaseConfiguration.create();
-
-    HColumnDescriptor hcd;
-    hcd = new HColumnDescriptor(family);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(100);
-    hcd.setMaxVersions(4);
-    init(name.getMethodName(), conf, hcd, false);
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true).setMobThreshold(100)
+            .setMaxVersions(4).build();
+    init(name.getMethodName(), conf, cfd, false);
 
     //Put data in memstore
     this.store.add(new KeyValue(row, family, qf1, 1, value), null);
@@ -503,15 +498,12 @@ public class TestHMobStore {
     String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
     Key cfKey = new SecretKeySpec(keyBytes, algorithm);
 
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(100);
-    hcd.setMaxVersions(4);
-    hcd.setEncryptionType(algorithm);
-    hcd.setEncryptionKey(EncryptionUtil.wrapKey(conf,
-      conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()),cfKey));
-
-    init(name.getMethodName(), conf, hcd, false);
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true).setMobThreshold(100)
+            .setMaxVersions(4).setEncryptionType(algorithm).setEncryptionKey(EncryptionUtil
+            .wrapKey(conf, conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
+                User.getCurrent().getShortName()), cfKey)).build();
+    init(name.getMethodName(), conf, cfd, false);
 
     this.store.add(new KeyValue(row, family, qf1, 1, value), null);
     this.store.add(new KeyValue(row, family, qf2, 1, value), null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
index 30ee3b2..ac63974 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -91,7 +92,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestHStoreFile.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private CacheConfig cacheConf =  new CacheConfig(TEST_UTIL.getConfiguration());
+  private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
   private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestStoreFile").toString();
   private static final ChecksumType CKTYPE = ChecksumType.CRC32C;
   private static final int CKBYTES = 512;
@@ -934,8 +935,7 @@ public class TestHStoreFile extends HBaseTestCase {
     Path baseDir = new Path(new Path(testDir, "7e0102"),"twoCOWEOC");
 
     // Grab the block cache and get the initial hit/miss counts
-    CacheConfig.instantiateBlockCache(conf);
-    BlockCache bc = new CacheConfig(conf).getBlockCache();
+    BlockCache bc = BlockCacheFactory.createBlockCache(conf);
     assertNotNull(bc);
     CacheStats cs = bc.getStats();
     long startHit = cs.getHitCount();
@@ -944,7 +944,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Let's write a StoreFile with three blocks, with cache on write off
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
-    CacheConfig cacheConf = new CacheConfig(conf);
+    CacheConfig cacheConf = new CacheConfig(conf, bc);
     Path pathCowOff = new Path(baseDir, "123456789");
     StoreFileWriter writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
     HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
@@ -967,7 +967,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Now write a StoreFile with three blocks, with cache on write on
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, bc);
     Path pathCowOn = new Path(baseDir, "123456788");
     writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
     hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
@@ -1025,7 +1025,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Let's close the first file with evict on close turned on
     conf.setBoolean("hbase.rs.evictblocksonclose", true);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, bc);
     hsf = new HStoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
     hsf.initReader();
     reader = hsf.getReader();
@@ -1039,7 +1039,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Let's close the second file with evict on close turned off
     conf.setBoolean("hbase.rs.evictblocksonclose", false);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, bc);
     hsf = new HStoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
     hsf.initReader();
     reader = hsf.getReader();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 cc9e385..cccc970 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
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,6 +50,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -58,14 +61,13 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -101,16 +103,6 @@ public class TestMobStoreCompaction {
   private final byte[] STARTROW = Bytes.toBytes(START_KEY);
   private int compactionThreshold;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    UTIL.startMiniCluster(1);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
   private void init(Configuration conf, long mobThreshold) throws Exception {
     this.conf = conf;
     this.mobCellThreshold = mobThreshold;
@@ -124,7 +116,9 @@ public class TestMobStoreCompaction {
     hcd.setMaxVersions(1);
     htd.modifyFamily(hcd);
 
-    region = UTIL.createLocalHRegion(htd, null, null);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    region = HBaseTestingUtility
+        .createRegionAndWAL(regionInfo, UTIL.getDataTestDir(), conf, htd, new MobFileCache(conf));
     fs = FileSystem.get(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
index bb97c9c..ab591ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
@@ -36,15 +36,16 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
@@ -139,12 +140,10 @@ public abstract class TestMultiColumnScanner {
     TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
     TEST_UTIL.getConfiguration().set(BloomFilterUtil.DELIMITER_KEY, "#");
     HRegion region = TEST_UTIL.createTestRegion(TABLE_NAME,
-        new HColumnDescriptor(FAMILY)
-            .setCompressionType(comprAlgo)
-            .setBloomFilterType(bloomType)
-            .setMaxVersions(MAX_VERSIONS)
-            .setDataBlockEncoding(dataBlockEncoding)
-    );
+        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_BYTES).setCompressionType(comprAlgo)
+            .setBloomFilterType(bloomType).setMaxVersions(MAX_VERSIONS)
+            .setDataBlockEncoding(dataBlockEncoding).build(),
+        BlockCacheFactory.createBlockCache(TEST_UTIL.getConfiguration()));
     List<String> rows = sequentialStrings("row", NUM_ROWS);
     List<String> qualifiers = sequentialStrings("qual", NUM_COLUMNS);
     List<KeyValue> kvs = new ArrayList<>();
@@ -202,7 +201,7 @@ public abstract class TestMultiColumnScanner {
     for (int maxVersions = 1; maxVersions <= TIMESTAMPS.length; ++maxVersions) {
       for (int columnBitMask = 1; columnBitMask <= MAX_COLUMN_BIT_MASK; ++columnBitMask) {
         Scan scan = new Scan();
-        scan.setMaxVersions(maxVersions);
+        scan.readVersions(maxVersions);
         Set<String> qualSet = new TreeSet<>();
         {
           int columnMaskTmp = columnBitMask;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
index 17b01b5..fd3a56d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.List;
+import java.util.Optional;
+
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
@@ -28,7 +30,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.ipc.MetricsHBaseServer;
 import org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapperStub;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -102,11 +103,9 @@ public class TestRSStatusServlet {
     Mockito.doReturn("fakequorum").when(zkw).getQuorum();
     Mockito.doReturn(zkw).when(rs).getZooKeeper();
 
-    // Fake CacheConfig
+    // Fake BlockCache
     LOG.warn("The " + HConstants.HFILE_BLOCK_CACHE_SIZE_KEY + " is set to 0");
-    CacheConfig cacheConf = Mockito.mock(CacheConfig.class);
-    Mockito.doReturn(null).when(cacheConf).getBlockCache();
-    Mockito.doReturn(cacheConf).when(rs).getCacheConfig();
+    Mockito.doReturn(Optional.empty()).when(rs).getBlockCache();
 
     // Fake MasterAddressTracker
     MasterAddressTracker mat = Mockito.mock(MasterAddressTracker.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 543126e..f8bd48c 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
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,16 +32,19 @@ import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -49,6 +53,7 @@ import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -69,8 +74,16 @@ public class TestRecoveredEdits {
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final Logger LOG = LoggerFactory.getLogger(TestRecoveredEdits.class);
+
+  private static BlockCache blockCache;
+
   @Rule public TestName testName = new TestName();
 
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    blockCache = BlockCacheFactory.createBlockCache(TEST_UTIL.getConfiguration());
+  }
+
   /**
    * HBASE-12782 ITBLL fails for me if generator does anything but 5M per maptask.
    * Create a region. Close it. Then copy into place a file to replay, one that is bigger than
@@ -81,7 +94,6 @@ public class TestRecoveredEdits {
   @Test
   public void testReplayWorksThoughLotsOfFlushing() throws
       IOException {
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
     for(MemoryCompactionPolicy policy : MemoryCompactionPolicy.values()) {
       testReplayWorksWithMemoryCompactionPolicy(policy);
     }
@@ -96,18 +108,22 @@ public class TestRecoveredEdits {
     // The file of recovered edits has a column family of 'meta'. Also has an encoded regionname
     // of 4823016d8fca70b25503ee07f4c6d79f which needs to match on replay.
     final String encodedRegionName = "4823016d8fca70b25503ee07f4c6d79f";
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(testName.getMethodName()));
     final String columnFamily = "meta";
     byte [][] columnFamilyAsByteArray = new byte [][] {Bytes.toBytes(columnFamily)};
-    htd.addFamily(new HColumnDescriptor(columnFamily));
-    HRegionInfo hri = new HRegionInfo(htd.getTableName()) {
+    TableDescriptor tableDescriptor =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(testName.getMethodName()))
+            .setColumnFamily(
+                ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(columnFamily)).build())
+            .build();
+    RegionInfo hri = new HRegionInfo(tableDescriptor.getTableName()) {
       @Override
       public synchronized String getEncodedName() {
         return encodedRegionName;
       }
 
       // Cache the name because lots of lookups.
-      private byte [] encodedRegionNameAsBytes = null;
+      private byte[] encodedRegionNameAsBytes = null;
+
       @Override
       public synchronized byte[] getEncodedNameAsBytes() {
         if (encodedRegionNameAsBytes == null) {
@@ -117,16 +133,16 @@ 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());
+    Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableDescriptor.getTableName());
     HRegionFileSystem hrfs =
         new HRegionFileSystem(TEST_UTIL.getConfiguration(), fs, tableDir, hri);
     if (fs.exists(hrfs.getRegionDir())) {
       LOG.info("Region directory already exists. Deleting.");
       fs.delete(hrfs.getRegionDir(), true);
     }
-    HRegion region = HRegion.createHRegion(hri, hbaseRootDir, conf, htd, null);
+    HRegion region = HBaseTestingUtility
+        .createRegionAndWAL(hri, hbaseRootDir, conf, tableDescriptor, blockCache);
     assertEquals(encodedRegionName, region.getRegionInfo().getEncodedName());
     List<String> storeFiles = region.getStoreFileList(columnFamilyAsByteArray);
     // There should be no store files.

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
index 61de21f..647f450 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
@@ -68,7 +68,7 @@ public class TestRowPrefixBloomFilter {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestRowPrefixBloomFilter.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private CacheConfig cacheConf =  new CacheConfig(TEST_UTIL.getConfiguration());
+  private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
   private static final ChecksumType CKTYPE = ChecksumType.CRC32C;
   private static final int CKBYTES = 512;
   private boolean localfs = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
index 75ebfd3..eb25806 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
@@ -214,7 +214,7 @@ public class TestSecureBulkLoadManager {
     ColumnFamilyDescriptor family = desc.getColumnFamily(FAMILY);
     Compression.Algorithm compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
 
-    CacheConfig writerCacheConf = new CacheConfig(conf, family);
+    CacheConfig writerCacheConf = new CacheConfig(conf, family, null);
     writerCacheConf.setCacheDataOnWrite(false);
     HFileContext hFileContext = new HFileContextBuilder()
         .withIncludesMvcc(false)


[41/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2

Posted by zh...@apache.org.
HBASE-21650 Add DDL operation and some other miscellaneous to thrift2


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

Branch: refs/heads/HBASE-21512
Commit: 7820ba1dbdba58b1002cdfde08eb21aa7a0bb6da
Parents: f5ea00f
Author: Allan Yang <al...@apache.org>
Authored: Thu Dec 27 22:25:33 2018 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Thu Dec 27 22:25:33 2018 +0800

----------------------------------------------------------------------
 .../hbase/thrift/generated/AlreadyExists.java   |     2 +-
 .../hbase/thrift/generated/BatchMutation.java   |     2 +-
 .../thrift/generated/ColumnDescriptor.java      |     2 +-
 .../hadoop/hbase/thrift/generated/Hbase.java    |     2 +-
 .../hadoop/hbase/thrift/generated/IOError.java  |     2 +-
 .../hbase/thrift/generated/IllegalArgument.java |     2 +-
 .../hadoop/hbase/thrift/generated/Mutation.java |     2 +-
 .../hadoop/hbase/thrift/generated/TAppend.java  |     2 +-
 .../hadoop/hbase/thrift/generated/TCell.java    |     2 +-
 .../hadoop/hbase/thrift/generated/TColumn.java  |     2 +-
 .../hbase/thrift/generated/TIncrement.java      |     2 +-
 .../hbase/thrift/generated/TRegionInfo.java     |     2 +-
 .../hbase/thrift/generated/TRowResult.java      |     2 +-
 .../hadoop/hbase/thrift/generated/TScan.java    |     2 +-
 .../thrift2/ThriftHBaseServiceHandler.java      |   290 +
 .../hadoop/hbase/thrift2/ThriftUtilities.java   |   411 +-
 .../thrift2/generated/NamespaceDescriptor.java  |   554 +
 .../hadoop/hbase/thrift2/generated/TAppend.java |   114 +-
 .../hbase/thrift2/generated/TAuthorization.java |     2 +-
 .../thrift2/generated/TBloomFilterType.java     |    69 +
 .../thrift2/generated/TCellVisibility.java      |     2 +-
 .../hadoop/hbase/thrift2/generated/TColumn.java |     2 +-
 .../generated/TColumnFamilyDescriptor.java      |  2519 +
 .../thrift2/generated/TColumnIncrement.java     |     2 +-
 .../hbase/thrift2/generated/TColumnValue.java   |   110 +-
 .../generated/TCompressionAlgorithm.java        |    60 +
 .../thrift2/generated/TDataBlockEncoding.java   |    57 +
 .../hadoop/hbase/thrift2/generated/TDelete.java |     2 +-
 .../hbase/thrift2/generated/TDurability.java    |     3 +
 .../hadoop/hbase/thrift2/generated/TGet.java    |   410 +-
 .../hbase/thrift2/generated/THBaseService.java  | 44644 +++++++++++++----
 .../hbase/thrift2/generated/THRegionInfo.java   |     2 +-
 .../thrift2/generated/THRegionLocation.java     |     2 +-
 .../hbase/thrift2/generated/TIOError.java       |     2 +-
 .../thrift2/generated/TIllegalArgument.java     |     2 +-
 .../hbase/thrift2/generated/TIncrement.java     |   114 +-
 .../thrift2/generated/TKeepDeletedCells.java    |    63 +
 .../thrift2/generated/TNamespaceDescriptor.java |   554 +
 .../hadoop/hbase/thrift2/generated/TPut.java    |     2 +-
 .../hadoop/hbase/thrift2/generated/TResult.java |   112 +-
 .../hbase/thrift2/generated/TRowMutations.java  |    38 +-
 .../hadoop/hbase/thrift2/generated/TScan.java   |     2 +-
 .../hbase/thrift2/generated/TServerName.java    |     2 +-
 .../thrift2/generated/TTableDescriptor.java     |   843 +
 .../hbase/thrift2/generated/TTableName.java     |   512 +
 .../hbase/thrift2/generated/TTimeRange.java     |     2 +-
 .../apache/hadoop/hbase/thrift2/hbase.thrift    |   229 +-
 .../thrift2/TestThriftHBaseServiceHandler.java  |    96 +
 48 files changed, 41553 insertions(+), 10303 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
index 68361c1..8ec3e32 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class AlreadyExists extends TException implements org.apache.thrift.TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
index a2920ea..0872223 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A BatchMutation object is used to apply a number of Mutations to a single row.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class BatchMutation implements org.apache.thrift.TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
index 3252377..bccd48b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * such as the number of versions, compression settings, etc. It is
  * used as input when creating a table or adding a column.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class ColumnDescriptor implements org.apache.thrift.TBase<ColumnDescriptor, ColumnDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
index f77ce14..dc4d8bc 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class Hbase {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
index 558f3e1..a32b008 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * to the Hbase master or an Hbase region server.  Also used to return
  * more general Hbase error conditions.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class IOError extends TException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable, Comparable<IOError> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
index bd296cc..573c496 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * An IllegalArgument exception indicates an illegal or invalid
  * argument was passed into a procedure.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class IllegalArgument extends TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
index 216df2d..72925b7 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A Mutation object is used to either update or delete a column-value.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class Mutation implements org.apache.thrift.TBase<Mutation, Mutation._Fields>, java.io.Serializable, Cloneable, Comparable<Mutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
index 439d71e..a21b5f7 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * An Append object is used to specify the parameters for performing the append operation.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
index 37021c0..afe0af1 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
  * the timestamp of a cell to a first-class value, making it easy to take
  * note of temporal data. Cell is used all the way from HStore up to HTable.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TCell implements org.apache.thrift.TBase<TCell, TCell._Fields>, java.io.Serializable, Cloneable, Comparable<TCell> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
index 77e875d..0115bbd 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Holds column name and the cell.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
index 22b5f79..7c018f1 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * For increments that are not incrementColumnValue
  * equivalents.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
index a1dab6b..ec75247 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A TRegionInfo contains information about an HTable region.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TRegionInfo implements org.apache.thrift.TBase<TRegionInfo, TRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TRegionInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
index 418f503..b09a368 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Holds row name and then a map of columns to cells.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResult._Fields>, java.io.Serializable, Cloneable, Comparable<TRowResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
index 3faadd9..97d59c7 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A Scan object is used to specify scanner parameters when opening a scanner.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index 8e3ee96..2bfeefe 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -19,18 +19,28 @@
 package org.apache.hadoop.hbase.thrift2;
 
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.appendFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.columnFamilyDescriptorFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.compareOpFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deletesFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getsFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.namespaceDescriptorFromHBase;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.namespaceDescriptorFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.namespaceDescriptorsFromHBase;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putsFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultFromHBase;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultsFromHBase;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.rowMutationsFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.scanFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.splitKeyFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableDescriptorFromHBase;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableDescriptorFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableDescriptorsFromHBase;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableNameFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.tableNamesFromHBase;
 import static org.apache.thrift.TBaseHelper.byteBufferToByteArray;
 
 import java.io.IOException;
@@ -45,16 +55,22 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.thrift.ThriftMetrics;
 import org.apache.hadoop.hbase.thrift2.generated.TAppend;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
 import org.apache.hadoop.hbase.thrift2.generated.TDelete;
 import org.apache.hadoop.hbase.thrift2.generated.TGet;
@@ -63,10 +79,13 @@ import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
 import org.apache.hadoop.hbase.thrift2.generated.TIOError;
 import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument;
 import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
+import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TPut;
 import org.apache.hadoop.hbase.thrift2.generated.TResult;
 import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
 import org.apache.hadoop.hbase.thrift2.generated.TScan;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ConnectionCache;
 import org.apache.thrift.TException;
@@ -570,4 +589,275 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
   private boolean isReadOnly() {
     return isReadOnly;
   }
+
+  @Override
+  public TTableDescriptor getTableDescriptor(TTableName table) throws TIOError, TException {
+    try {
+      TableName tableName = ThriftUtilities.tableNameFromThrift(table);
+      TableDescriptor tableDescriptor = connectionCache.getAdmin().getDescriptor(tableName);
+      return tableDescriptorFromHBase(tableDescriptor);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public List<TTableDescriptor> getTableDescriptors(List<TTableName> tables)
+      throws TIOError, TException {
+    try {
+      List<TableName> tableNames = ThriftUtilities.tableNamesFromThrift(tables);
+      List<TableDescriptor> tableDescriptors = connectionCache.getAdmin()
+          .listTableDescriptors(tableNames);
+      return tableDescriptorsFromHBase(tableDescriptors);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(TTableName tTableName) throws TIOError, TException {
+    try {
+      TableName tableName = tableNameFromThrift(tTableName);
+      return connectionCache.getAdmin().tableExists(tableName);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public List<TTableDescriptor> getTableDescriptorsByPattern(String regex, boolean includeSysTables)
+      throws TIOError, TException {
+    try {
+      Pattern pattern = Pattern.compile(regex);
+      List<TableDescriptor> tableDescriptors = connectionCache.getAdmin()
+          .listTableDescriptors(pattern, includeSysTables);
+      return tableDescriptorsFromHBase(tableDescriptors);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public List<TTableDescriptor> getTableDescriptorsByNamespace(String name)
+      throws TIOError, TException {
+    try {
+      List<TableDescriptor> descriptors = connectionCache.getAdmin()
+          .listTableDescriptorsByNamespace(Bytes.toBytes(name));
+      return tableDescriptorsFromHBase(descriptors);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public List<TTableName> getTableNamesByPattern(String regex, boolean includeSysTables)
+      throws TIOError, TException {
+    try {
+      Pattern pattern = Pattern.compile(regex);
+      TableName[] tableNames = connectionCache.getAdmin()
+          .listTableNames(pattern, includeSysTables);
+      return tableNamesFromHBase(tableNames);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public List<TTableName> getTableNamesByNamespace(String name) throws TIOError, TException {
+    try {
+      TableName[] tableNames = connectionCache.getAdmin().listTableNamesByNamespace(name);
+      return tableNamesFromHBase(tableNames);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void createTable(TTableDescriptor desc, List<ByteBuffer> splitKeys)
+      throws TIOError, TException {
+    try {
+      TableDescriptor descriptor = tableDescriptorFromThrift(desc);
+      byte[][] split = splitKeyFromThrift(splitKeys);
+      connectionCache.getAdmin().createTable(descriptor, split);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void deleteTable(TTableName tableName) throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      connectionCache.getAdmin().deleteTable(table);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void truncateTable(TTableName tableName, boolean preserveSplits)
+      throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      connectionCache.getAdmin().truncateTable(table, preserveSplits);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void enableTable(TTableName tableName) throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      connectionCache.getAdmin().enableTable(table);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void disableTable(TTableName tableName) throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      connectionCache.getAdmin().disableTable(table);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public boolean isTableEnabled(TTableName tableName) throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      return connectionCache.getAdmin().isTableEnabled(table);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public boolean isTableDisabled(TTableName tableName) throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      return connectionCache.getAdmin().isTableDisabled(table);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public boolean isTableAvailable(TTableName tableName) throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      return connectionCache.getAdmin().isTableAvailable(table);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public boolean isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys)
+      throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      byte[][] split = splitKeyFromThrift(splitKeys);
+      return connectionCache.getAdmin().isTableAvailable(table, split);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void addColumnFamily(TTableName tableName, TColumnFamilyDescriptor column)
+      throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptorFromThrift(column);
+      connectionCache.getAdmin().addColumnFamily(table, columnFamilyDescriptor);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void deleteColumnFamily(TTableName tableName, ByteBuffer column)
+      throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      connectionCache.getAdmin().deleteColumnFamily(table, column.array());
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void modifyColumnFamily(TTableName tableName, TColumnFamilyDescriptor column)
+      throws TIOError, TException {
+    try {
+      TableName table = tableNameFromThrift(tableName);
+      ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptorFromThrift(column);
+      connectionCache.getAdmin().modifyColumnFamily(table, columnFamilyDescriptor);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void modifyTable(TTableDescriptor desc) throws TIOError, TException {
+    try {
+      TableDescriptor descriptor = tableDescriptorFromThrift(desc);
+      connectionCache.getAdmin().modifyTable(descriptor);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void createNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, TException {
+    try {
+      NamespaceDescriptor descriptor = namespaceDescriptorFromThrift(namespaceDesc);
+      connectionCache.getAdmin().createNamespace(descriptor);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void modifyNamespace(TNamespaceDescriptor namespaceDesc) throws TIOError, TException {
+    try {
+      NamespaceDescriptor descriptor = namespaceDescriptorFromThrift(namespaceDesc);
+      connectionCache.getAdmin().modifyNamespace(descriptor);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public void deleteNamespace(String name) throws TIOError, TException {
+    try {
+      connectionCache.getAdmin().deleteNamespace(name);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public TNamespaceDescriptor getNamespaceDescriptor(String name) throws TIOError, TException {
+    try {
+      NamespaceDescriptor descriptor = connectionCache.getAdmin().getNamespaceDescriptor(name);
+      return namespaceDescriptorFromHBase(descriptor);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
+
+  @Override
+  public List<TNamespaceDescriptor> listNamespaceDescriptors() throws TIOError, TException {
+    try {
+      NamespaceDescriptor[] descriptors = connectionCache.getAdmin().listNamespaceDescriptors();
+      return namespaceDescriptorsFromHBase(descriptors);
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
index 9b42bce..63c7007 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
@@ -34,9 +34,14 @@ import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.KeepDeletedCells;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -48,28 +53,42 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Scan.ReadType;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.filter.ParseFilter;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.thrift2.generated.TAppend;
+import org.apache.hadoop.hbase.thrift2.generated.TBloomFilterType;
 import org.apache.hadoop.hbase.thrift2.generated.TColumn;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
 import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
+import org.apache.hadoop.hbase.thrift2.generated.TCompressionAlgorithm;
 import org.apache.hadoop.hbase.thrift2.generated.TConsistency;
+import org.apache.hadoop.hbase.thrift2.generated.TDataBlockEncoding;
 import org.apache.hadoop.hbase.thrift2.generated.TDelete;
+import org.apache.hadoop.hbase.thrift2.generated.TDeleteType;
 import org.apache.hadoop.hbase.thrift2.generated.TDurability;
 import org.apache.hadoop.hbase.thrift2.generated.TGet;
 import org.apache.hadoop.hbase.thrift2.generated.THRegionInfo;
 import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
 import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
+import org.apache.hadoop.hbase.thrift2.generated.TKeepDeletedCells;
 import org.apache.hadoop.hbase.thrift2.generated.TMutation;
+import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TPut;
 import org.apache.hadoop.hbase.thrift2.generated.TReadType;
 import org.apache.hadoop.hbase.thrift2.generated.TResult;
 import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
 import org.apache.hadoop.hbase.thrift2.generated.TScan;
 import org.apache.hadoop.hbase.thrift2.generated.TServerName;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
 import org.apache.hadoop.hbase.thrift2.generated.TTimeRange;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -79,6 +98,12 @@ import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils;
 @InterfaceAudience.Private
 public final class ThriftUtilities {
 
+  private final static Cell[] EMPTY_CELL_ARRAY = new Cell[]{};
+  private final static Result EMPTY_RESULT = Result.create(EMPTY_CELL_ARRAY);
+  private final static Result EMPTY_RESULT_STALE = Result.create(EMPTY_CELL_ARRAY, null, true);
+
+
+
   private ThriftUtilities() {
     throw new UnsupportedOperationException("Can't initialize class");
   }
@@ -129,6 +154,20 @@ public final class ThriftUtilities {
       out.setReplicaId(in.getTargetReplicaId());
     }
 
+    if (in.isSetCacheBlocks()) {
+      out.setCacheBlocks(in.isCacheBlocks());
+    }
+    if (in.isSetStoreLimit()) {
+      out.setMaxResultsPerColumnFamily(in.getStoreLimit());
+    }
+    if (in.isSetStoreOffset()) {
+      out.setRowOffsetPerColumnFamily(in.getStoreOffset());
+    }
+    if (in.isSetExistence_only()) {
+      out.setCheckExistenceOnly(in.isExistence_only());
+    }
+
+
     if (!in.isSetColumns()) {
       return out;
     }
@@ -183,6 +222,7 @@ public final class ThriftUtilities {
       col.setQualifier(CellUtil.cloneQualifier(kv));
       col.setTimestamp(kv.getTimestamp());
       col.setValue(CellUtil.cloneValue(kv));
+      col.setType(kv.getType().getCode());
       if (kv.getTagsLength() > 0) {
         col.setTags(PrivateCellUtil.cloneTags(kv));
       }
@@ -191,6 +231,8 @@ public final class ThriftUtilities {
     out.setColumnValues(columnValues);
 
     out.setStale(in.isStale());
+
+    out.setPartial(in.mayHaveMoreCellsInRow());
     return out;
   }
 
@@ -373,6 +415,15 @@ public final class ThriftUtilities {
     return out;
   }
 
+  public static TDeleteType deleteTypeFromHBase(Cell.Type type) {
+    switch (type) {
+      case Delete: return TDeleteType.DELETE_COLUMN;
+      case DeleteColumn: return TDeleteType.DELETE_COLUMNS;
+      case DeleteFamily: return TDeleteType.DELETE_FAMILY;
+      case DeleteFamilyVersion: return TDeleteType.DELETE_FAMILY_VERSION;
+      default: throw new IllegalArgumentException("Unknow delete type " + type);
+    }  }
+
   public static TDelete deleteFromHBase(Delete in) {
     TDelete out = new TDelete(ByteBuffer.wrap(in.getRow()));
 
@@ -382,17 +433,33 @@ public final class ThriftUtilities {
       out.setTimestamp(rowTimestamp);
     }
 
-    // Map<family, List<KeyValue>>
-    for (Map.Entry<byte[], List<org.apache.hadoop.hbase.Cell>> familyEntry:
+    for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
+      out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
+          ByteBuffer.wrap(attribute.getValue()));
+    }
+    if (in.getDurability() != Durability.USE_DEFAULT)  {
+      out.setDurability(durabilityFromHBase(in.getDurability()));
+    }
+    // Delete the whole row
+    if (in.getFamilyCellMap().size() == 0) {
+      return out;
+    }
+    TDeleteType type = null;
+    for (Map.Entry<byte[], List<Cell>> familyEntry:
         in.getFamilyCellMap().entrySet()) {
+      byte[] family = familyEntry.getKey();
       TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey()));
-      for (org.apache.hadoop.hbase.Cell cell: familyEntry.getValue()) {
-        byte[] family = CellUtil.cloneFamily(cell);
+      for (Cell cell: familyEntry.getValue()) {
+        TDeleteType cellDeleteType = deleteTypeFromHBase(cell.getType());
+        if (type == null) {
+          type = cellDeleteType;
+        } else if (type != cellDeleteType){
+          throw new RuntimeException("Only the same delete type is supported, but two delete type "
+              + "is founded, one is " + type + " the other one is " + cellDeleteType);
+        }
         byte[] qualifier = CellUtil.cloneQualifier(cell);
         long timestamp = cell.getTimestamp();
-        if (family != null) {
-          column.setFamily(family);
-        }
+        column.setFamily(family);
         if (qualifier != null) {
           column.setQualifier(qualifier);
         }
@@ -403,6 +470,7 @@ public final class ThriftUtilities {
       columns.add(column);
     }
     out.setColumns(columns);
+    out.setDeleteType(type);
 
     return out;
   }
@@ -532,6 +600,10 @@ public final class ThriftUtilities {
       out.setCellVisibility(new CellVisibility(in.getCellVisibility().getExpression()));
     }
 
+    if (in.isSetReturnResults()) {
+      out.setReturnResults(in.isReturnResults());
+    }
+
     return out;
   }
 
@@ -553,6 +625,10 @@ public final class ThriftUtilities {
       out.setCellVisibility(new CellVisibility(append.getCellVisibility().getExpression()));
     }
 
+    if (append.isSetReturnResults()) {
+      out.setReturnResults(append.isReturnResults());
+    }
+
     return out;
   }
 
@@ -606,11 +682,12 @@ public final class ThriftUtilities {
 
   private static Durability durabilityFromThrift(TDurability tDurability) {
     switch (tDurability.getValue()) {
+      case 0: return Durability.USE_DEFAULT;
       case 1: return Durability.SKIP_WAL;
       case 2: return Durability.ASYNC_WAL;
       case 3: return Durability.SYNC_WAL;
       case 4: return Durability.FSYNC_WAL;
-      default: return null;
+      default: return Durability.USE_DEFAULT;
     }
   }
 
@@ -643,4 +720,322 @@ public final class ThriftUtilities {
       default: return Consistency.STRONG;
     }
   }
+
+  public static TableName tableNameFromThrift(TTableName tableName) {
+    return TableName.valueOf(tableName.getNs(), tableName.getQualifier());
+  }
+
+  public static List<TableName> tableNamesFromThrift(List<TTableName> tableNames) {
+    List<TableName> out = new ArrayList<>(tableNames.size());
+    for (TTableName tableName : tableNames) {
+      out.add(tableNameFromThrift(tableName));
+    }
+    return out;
+  }
+
+  public static TTableName tableNameFromHBase(TableName table) {
+    TTableName tableName = new TTableName();
+    tableName.setNs(table.getNamespace());
+    tableName.setQualifier(table.getQualifier());
+    return tableName;
+  }
+
+  public static List<TTableName> tableNamesFromHBase(TableName[] in) {
+    List<TTableName> out = new ArrayList<>(in.length);
+    for (TableName tableName : in) {
+      out.add(tableNameFromHBase(tableName));
+    }
+    return out;
+  }
+
+  public static byte[][] splitKeyFromThrift(List<ByteBuffer> in) {
+    if (in == null || in.size() == 0) {
+      return null;
+    }
+    byte[][] out = new byte[in.size()][];
+    int index = 0;
+    for (ByteBuffer key : in) {
+      out[index++] = key.array();
+    }
+    return out;
+  }
+
+  public static BloomType bloomFilterFromThrift(TBloomFilterType in) {
+    switch (in.getValue()) {
+      case 0: return BloomType.NONE;
+      case 1: return BloomType.ROW;
+      case 2: return BloomType.ROWCOL;
+      case 3: return BloomType.ROWPREFIX_FIXED_LENGTH;
+      case 4: return BloomType.ROWPREFIX_DELIMITED;
+      default: return BloomType.ROW;
+    }
+  }
+
+  public static Compression.Algorithm compressionAlgorithmFromThrift(TCompressionAlgorithm in) {
+    switch (in.getValue()) {
+      case 0: return Compression.Algorithm.LZO;
+      case 1: return Compression.Algorithm.GZ;
+      case 2: return Compression.Algorithm.NONE;
+      case 3: return Compression.Algorithm.SNAPPY;
+      case 4: return Compression.Algorithm.LZ4;
+      case 5: return Compression.Algorithm.BZIP2;
+      case 6: return Compression.Algorithm.ZSTD;
+      default: return Compression.Algorithm.NONE;
+    }
+  }
+
+  public static DataBlockEncoding dataBlockEncodingFromThrift(TDataBlockEncoding in) {
+    switch (in.getValue()) {
+      case 0: return DataBlockEncoding.NONE;
+      case 2: return DataBlockEncoding.PREFIX;
+      case 3: return DataBlockEncoding.DIFF;
+      case 4: return DataBlockEncoding.FAST_DIFF;
+      case 7: return DataBlockEncoding.ROW_INDEX_V1;
+      default: return DataBlockEncoding.NONE;
+    }
+  }
+
+  public static KeepDeletedCells keepDeletedCellsFromThrift(TKeepDeletedCells in) {
+    switch (in.getValue()) {
+      case 0: return KeepDeletedCells.FALSE;
+      case 1: return KeepDeletedCells.TRUE;
+      case 2: return KeepDeletedCells.TTL;
+      default: return KeepDeletedCells.FALSE;
+    }
+  }
+
+  public static ColumnFamilyDescriptor columnFamilyDescriptorFromThrift(
+      TColumnFamilyDescriptor in) {
+    ColumnFamilyDescriptorBuilder builder = ColumnFamilyDescriptorBuilder
+        .newBuilder(in.getName());
+
+    if (in.isSetAttributes()) {
+      for (Map.Entry<ByteBuffer, ByteBuffer> attribute : in.getAttributes().entrySet()) {
+        builder.setValue(attribute.getKey().array(), attribute.getValue().array());
+      }
+    }
+    if (in.isSetConfiguration()) {
+      for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
+        builder.setConfiguration(conf.getKey(), conf.getValue());
+      }
+    }
+    if (in.isSetBlockSize()) {
+      builder.setBlocksize(in.getBlockSize());
+    }
+    if (in.isSetBloomnFilterType()) {
+      builder.setBloomFilterType(bloomFilterFromThrift(in.getBloomnFilterType()));
+    }
+    if (in.isSetCompressionType()) {
+      builder.setCompressionType(compressionAlgorithmFromThrift(in.getCompressionType()));
+    }
+    if (in.isSetDfsReplication()) {
+      builder.setDFSReplication(in.getDfsReplication());
+    }
+    if (in.isSetDataBlockEncoding()) {
+      builder.setDataBlockEncoding(dataBlockEncodingFromThrift(in.getDataBlockEncoding()));
+    }
+    if (in.isSetKeepDeletedCells()) {
+      builder.setKeepDeletedCells(keepDeletedCellsFromThrift(in.getKeepDeletedCells()));
+    }
+    if (in.isSetMaxVersions()) {
+      builder.setMaxVersions(in.getMaxVersions());
+    }
+    if (in.isSetMinVersions()) {
+      builder.setMinVersions(in.getMinVersions());
+    }
+    if (in.isSetScope()) {
+      builder.setScope(in.getScope());
+    }
+    if (in.isSetTimeToLive()) {
+      builder.setTimeToLive(in.getTimeToLive());
+    }
+    if (in.isSetBlockCacheEnabled()) {
+      builder.setBlockCacheEnabled(in.isBlockCacheEnabled());
+    }
+    if (in.isSetCacheBloomsOnWrite()) {
+      builder.setCacheBloomsOnWrite(in.isCacheBloomsOnWrite());
+    }
+    if (in.isSetCacheDataOnWrite()) {
+      builder.setCacheDataOnWrite(in.isCacheDataOnWrite());
+    }
+    if (in.isSetCacheIndexesOnWrite()) {
+      builder.setCacheIndexesOnWrite(in.isCacheIndexesOnWrite());
+    }
+    if (in.isSetCompressTags()) {
+      builder.setCompressTags(in.isCompressTags());
+    }
+    if (in.isSetEvictBlocksOnClose()) {
+      builder.setEvictBlocksOnClose(in.isEvictBlocksOnClose());
+    }
+    if (in.isSetInMemory()) {
+      builder.setInMemory(in.isInMemory());
+    }
+
+
+    return builder.build();
+  }
+
+  public static NamespaceDescriptor namespaceDescriptorFromThrift(TNamespaceDescriptor in) {
+    NamespaceDescriptor.Builder builder = NamespaceDescriptor.create(in.getName());
+    if (in.isSetConfiguration()) {
+      for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
+        builder.addConfiguration(conf.getKey(), conf.getValue());
+      }
+    }
+    return builder.build();
+  }
+
+  public static TNamespaceDescriptor namespaceDescriptorFromHBase(NamespaceDescriptor in) {
+    TNamespaceDescriptor out = new TNamespaceDescriptor();
+    out.setName(in.getName());
+    for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
+      out.putToConfiguration(conf.getKey(), conf.getValue());
+    }
+    return out;
+  }
+
+  public static List<TNamespaceDescriptor> namespaceDescriptorsFromHBase(
+      NamespaceDescriptor[] in) {
+    List<TNamespaceDescriptor> out = new ArrayList<>(in.length);
+    for (NamespaceDescriptor descriptor : in) {
+      out.add(namespaceDescriptorFromHBase(descriptor));
+    }
+    return out;
+  }
+
+  public static TableDescriptor tableDescriptorFromThrift(TTableDescriptor in) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder
+        .newBuilder(tableNameFromThrift(in.getTableName()));
+    for (TColumnFamilyDescriptor column : in.getColumns()) {
+      builder.setColumnFamily(columnFamilyDescriptorFromThrift(column));
+    }
+    if (in.isSetAttributes()) {
+      for (Map.Entry<ByteBuffer, ByteBuffer> attribute : in.getAttributes().entrySet()) {
+        builder.setValue(attribute.getKey().array(), attribute.getValue().array());
+      }
+    }
+    if (in.isSetDurability()) {
+      builder.setDurability(durabilityFromThrift(in.getDurability()));
+    }
+    return builder.build();
+  }
+
+  private static TDurability durabilityFromHBase(Durability durability) {
+    switch (durability) {
+      case USE_DEFAULT: return TDurability.USE_DEFAULT;
+      case SKIP_WAL: return TDurability.SKIP_WAL;
+      case ASYNC_WAL: return TDurability.ASYNC_WAL;
+      case SYNC_WAL: return TDurability.SYNC_WAL;
+      case FSYNC_WAL: return TDurability.FSYNC_WAL;
+      default: return null;
+    }
+  }
+
+  public static TTableDescriptor tableDescriptorFromHBase(TableDescriptor in) {
+    TTableDescriptor out = new TTableDescriptor();
+    out.setTableName(tableNameFromHBase(in.getTableName()));
+    Map<Bytes, Bytes> attributes = in.getValues();
+    for (Map.Entry<Bytes, Bytes> attribute : attributes.entrySet()) {
+      out.putToAttributes(ByteBuffer.wrap(attribute.getKey().get()),
+          ByteBuffer.wrap(attribute.getValue().get()));
+    }
+    for (ColumnFamilyDescriptor column : in.getColumnFamilies()) {
+      out.addToColumns(columnFamilyDescriptorFromHBase(column));
+    }
+    out.setDurability(durabilityFromHBase(in.getDurability()));
+    return out;
+  }
+
+  public static List<TTableDescriptor> tableDescriptorsFromHBase(List<TableDescriptor> in) {
+    List<TTableDescriptor> out = new ArrayList<>(in.size());
+    for (TableDescriptor descriptor : in) {
+      out.add(tableDescriptorFromHBase(descriptor));
+    }
+    return out;
+  }
+
+  public static List<TTableDescriptor> tableDescriptorsFromHBase(TableDescriptor[] in) {
+    List<TTableDescriptor> out = new ArrayList<>(in.length);
+    for (TableDescriptor descriptor : in) {
+      out.add(tableDescriptorFromHBase(descriptor));
+    }
+    return out;
+  }
+
+
+  public static TBloomFilterType bloomFilterFromHBase(BloomType in) {
+    switch (in) {
+      case NONE: return TBloomFilterType.NONE;
+      case ROW: return TBloomFilterType.ROW;
+      case ROWCOL: return TBloomFilterType.ROWCOL;
+      case ROWPREFIX_FIXED_LENGTH: return TBloomFilterType.ROWPREFIX_FIXED_LENGTH;
+      case ROWPREFIX_DELIMITED: return TBloomFilterType.ROWPREFIX_DELIMITED;
+      default: return TBloomFilterType.ROW;
+    }
+  }
+
+  public static TCompressionAlgorithm compressionAlgorithmFromHBase(Compression.Algorithm in) {
+    switch (in) {
+      case LZO: return TCompressionAlgorithm.LZO;
+      case GZ: return TCompressionAlgorithm.GZ;
+      case NONE: return TCompressionAlgorithm.NONE;
+      case SNAPPY: return TCompressionAlgorithm.SNAPPY;
+      case LZ4: return TCompressionAlgorithm.LZ4;
+      case BZIP2: return TCompressionAlgorithm.BZIP2;
+      case ZSTD: return TCompressionAlgorithm.ZSTD;
+      default: return TCompressionAlgorithm.NONE;
+    }
+  }
+
+  public static TDataBlockEncoding dataBlockEncodingFromHBase(DataBlockEncoding in) {
+    switch (in) {
+      case NONE: return TDataBlockEncoding.NONE;
+      case PREFIX: return TDataBlockEncoding.PREFIX;
+      case DIFF: return TDataBlockEncoding.DIFF;
+      case FAST_DIFF: return TDataBlockEncoding.FAST_DIFF;
+      case ROW_INDEX_V1: return TDataBlockEncoding.ROW_INDEX_V1;
+      default: return TDataBlockEncoding.NONE;
+    }
+  }
+
+  public static TKeepDeletedCells keepDeletedCellsFromHBase(KeepDeletedCells in) {
+    switch (in) {
+      case FALSE: return TKeepDeletedCells.FALSE;
+      case TRUE: return TKeepDeletedCells.TRUE;
+      case TTL: return TKeepDeletedCells.TTL;
+      default: return TKeepDeletedCells.FALSE;
+    }
+  }
+
+  public static TColumnFamilyDescriptor columnFamilyDescriptorFromHBase(
+      ColumnFamilyDescriptor in) {
+    TColumnFamilyDescriptor out = new TColumnFamilyDescriptor();
+    out.setName(in.getName());
+    for (Map.Entry<Bytes, Bytes> attribute : in.getValues().entrySet()) {
+      out.putToAttributes(ByteBuffer.wrap(attribute.getKey().get()),
+          ByteBuffer.wrap(attribute.getValue().get()));
+    }
+    for (Map.Entry<String, String> conf : in.getConfiguration().entrySet()) {
+      out.putToConfiguration(conf.getKey(), conf.getValue());
+    }
+    out.setBlockSize(in.getBlocksize());
+    out.setBloomnFilterType(bloomFilterFromHBase(in.getBloomFilterType()));
+    out.setCompressionType(compressionAlgorithmFromHBase(in.getCompressionType()));
+    out.setDfsReplication(in.getDFSReplication());
+    out.setDataBlockEncoding(dataBlockEncodingFromHBase(in.getDataBlockEncoding()));
+    out.setKeepDeletedCells(keepDeletedCellsFromHBase(in.getKeepDeletedCells()));
+    out.setMaxVersions(in.getMaxVersions());
+    out.setMinVersions(in.getMinVersions());
+    out.setScope(in.getScope());
+    out.setTimeToLive(in.getTimeToLive());
+    out.setBlockCacheEnabled(in.isBlockCacheEnabled());
+    out.setCacheBloomsOnWrite(in.isCacheBloomsOnWrite());
+    out.setCacheDataOnWrite(in.isCacheDataOnWrite());
+    out.setCacheIndexesOnWrite(in.isCacheIndexesOnWrite());
+    out.setCompressTags(in.isCompressTags());
+    out.setEvictBlocksOnClose(in.isEvictBlocksOnClose());
+    out.setInMemory(in.isInMemory());
+    return out;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/NamespaceDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/NamespaceDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/NamespaceDescriptor.java
new file mode 100644
index 0000000..3f9f512
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/NamespaceDescriptor.java
@@ -0,0 +1,554 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-26")
+public class NamespaceDescriptor implements org.apache.thrift.TBase<NamespaceDescriptor, NamespaceDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<NamespaceDescriptor> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NamespaceDescriptor");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NamespaceDescriptorStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NamespaceDescriptorTupleSchemeFactory());
+  }
+
+  public String name; // required
+  public Map<String,String> configuration; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    CONFIGURATION((short)2, "configuration");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // CONFIGURATION
+          return CONFIGURATION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CONFIGURATION};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NamespaceDescriptor.class, metaDataMap);
+  }
+
+  public NamespaceDescriptor() {
+  }
+
+  public NamespaceDescriptor(
+    String name)
+  {
+    this();
+    this.name = name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NamespaceDescriptor(NamespaceDescriptor other) {
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+    if (other.isSetConfiguration()) {
+      Map<String,String> __this__configuration = new HashMap<String,String>(other.configuration);
+      this.configuration = __this__configuration;
+    }
+  }
+
+  public NamespaceDescriptor deepCopy() {
+    return new NamespaceDescriptor(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.configuration = null;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public NamespaceDescriptor setName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int getConfigurationSize() {
+    return (this.configuration == null) ? 0 : this.configuration.size();
+  }
+
+  public void putToConfiguration(String key, String val) {
+    if (this.configuration == null) {
+      this.configuration = new HashMap<String,String>();
+    }
+    this.configuration.put(key, val);
+  }
+
+  public Map<String,String> getConfiguration() {
+    return this.configuration;
+  }
+
+  public NamespaceDescriptor setConfiguration(Map<String,String> configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  public void unsetConfiguration() {
+    this.configuration = null;
+  }
+
+  /** Returns true if field configuration is set (has been assigned a value) and false otherwise */
+  public boolean isSetConfiguration() {
+    return this.configuration != null;
+  }
+
+  public void setConfigurationIsSet(boolean value) {
+    if (!value) {
+      this.configuration = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    case CONFIGURATION:
+      if (value == null) {
+        unsetConfiguration();
+      } else {
+        setConfiguration((Map<String,String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    case CONFIGURATION:
+      return getConfiguration();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    case CONFIGURATION:
+      return isSetConfiguration();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NamespaceDescriptor)
+      return this.equals((NamespaceDescriptor)that);
+    return false;
+  }
+
+  public boolean equals(NamespaceDescriptor that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_configuration = true && this.isSetConfiguration();
+    boolean that_present_configuration = true && that.isSetConfiguration();
+    if (this_present_configuration || that_present_configuration) {
+      if (!(this_present_configuration && that_present_configuration))
+        return false;
+      if (!this.configuration.equals(that.configuration))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_configuration = true && (isSetConfiguration());
+    list.add(present_configuration);
+    if (present_configuration)
+      list.add(configuration);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(NamespaceDescriptor other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(other.isSetConfiguration());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetConfiguration()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, other.configuration);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NamespaceDescriptor(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (isSetConfiguration()) {
+      if (!first) sb.append(", ");
+      sb.append("configuration:");
+      if (this.configuration == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.configuration);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (name == null) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString());
+    }
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class NamespaceDescriptorStandardSchemeFactory implements SchemeFactory {
+    public NamespaceDescriptorStandardScheme getScheme() {
+      return new NamespaceDescriptorStandardScheme();
+    }
+  }
+
+  private static class NamespaceDescriptorStandardScheme extends StandardScheme<NamespaceDescriptor> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NamespaceDescriptor struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // CONFIGURATION
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map180 = iprot.readMapBegin();
+                struct.configuration = new HashMap<String,String>(2*_map180.size);
+                String _key181;
+                String _val182;
+                for (int _i183 = 0; _i183 < _map180.size; ++_i183)
+                {
+                  _key181 = iprot.readString();
+                  _val182 = iprot.readString();
+                  struct.configuration.put(_key181, _val182);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setConfigurationIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, NamespaceDescriptor struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.configuration != null) {
+        if (struct.isSetConfiguration()) {
+          oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size()));
+            for (Map.Entry<String, String> _iter184 : struct.configuration.entrySet())
+            {
+              oprot.writeString(_iter184.getKey());
+              oprot.writeString(_iter184.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NamespaceDescriptorTupleSchemeFactory implements SchemeFactory {
+    public NamespaceDescriptorTupleScheme getScheme() {
+      return new NamespaceDescriptorTupleScheme();
+    }
+  }
+
+  private static class NamespaceDescriptorTupleScheme extends TupleScheme<NamespaceDescriptor> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NamespaceDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.name);
+      BitSet optionals = new BitSet();
+      if (struct.isSetConfiguration()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetConfiguration()) {
+        {
+          oprot.writeI32(struct.configuration.size());
+          for (Map.Entry<String, String> _iter185 : struct.configuration.entrySet())
+          {
+            oprot.writeString(_iter185.getKey());
+            oprot.writeString(_iter185.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NamespaceDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.name = iprot.readString();
+      struct.setNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map186 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.configuration = new HashMap<String,String>(2*_map186.size);
+          String _key187;
+          String _val188;
+          for (int _i189 = 0; _i189 < _map186.size; ++_i189)
+          {
+            _key187 = iprot.readString();
+            _val188 = iprot.readString();
+            struct.configuration.put(_key187, _val188);
+          }
+        }
+        struct.setConfigurationIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
index e0e3074..c89f67b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 
@@ -43,6 +43,7 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
   private static final org.apache.thrift.protocol.TField ATTRIBUTES_FIELD_DESC = new org.apache.thrift.protocol.TField("attributes", org.apache.thrift.protocol.TType.MAP, (short)3);
   private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)4);
   private static final org.apache.thrift.protocol.TField CELL_VISIBILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("cellVisibility", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField RETURN_RESULTS_FIELD_DESC = new org.apache.thrift.protocol.TField("returnResults", org.apache.thrift.protocol.TType.BOOL, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -59,6 +60,7 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
    */
   public TDurability durability; // optional
   public TCellVisibility cellVisibility; // optional
+  public boolean returnResults; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -70,7 +72,8 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
      * @see TDurability
      */
     DURABILITY((short)4, "durability"),
-    CELL_VISIBILITY((short)5, "cellVisibility");
+    CELL_VISIBILITY((short)5, "cellVisibility"),
+    RETURN_RESULTS((short)6, "returnResults");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -95,6 +98,8 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
           return DURABILITY;
         case 5: // CELL_VISIBILITY
           return CELL_VISIBILITY;
+        case 6: // RETURN_RESULTS
+          return RETURN_RESULTS;
         default:
           return null;
       }
@@ -135,7 +140,9 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.DURABILITY,_Fields.CELL_VISIBILITY};
+  private static final int __RETURNRESULTS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.DURABILITY,_Fields.CELL_VISIBILITY,_Fields.RETURN_RESULTS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -152,6 +159,8 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
     tmpMap.put(_Fields.CELL_VISIBILITY, new org.apache.thrift.meta_data.FieldMetaData("cellVisibility", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCellVisibility.class)));
+    tmpMap.put(_Fields.RETURN_RESULTS, new org.apache.thrift.meta_data.FieldMetaData("returnResults", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAppend.class, metaDataMap);
   }
@@ -172,6 +181,7 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
    * Performs a deep copy on <i>other</i>.
    */
   public TAppend(TAppend other) {
+    __isset_bitfield = other.__isset_bitfield;
     if (other.isSetRow()) {
       this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row);
     }
@@ -192,6 +202,7 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
     if (other.isSetCellVisibility()) {
       this.cellVisibility = new TCellVisibility(other.cellVisibility);
     }
+    this.returnResults = other.returnResults;
   }
 
   public TAppend deepCopy() {
@@ -205,6 +216,8 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
     this.attributes = null;
     this.durability = null;
     this.cellVisibility = null;
+    setReturnResultsIsSet(false);
+    this.returnResults = false;
   }
 
   public byte[] getRow() {
@@ -371,6 +384,29 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
     }
   }
 
+  public boolean isReturnResults() {
+    return this.returnResults;
+  }
+
+  public TAppend setReturnResults(boolean returnResults) {
+    this.returnResults = returnResults;
+    setReturnResultsIsSet(true);
+    return this;
+  }
+
+  public void unsetReturnResults() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RETURNRESULTS_ISSET_ID);
+  }
+
+  /** Returns true if field returnResults is set (has been assigned a value) and false otherwise */
+  public boolean isSetReturnResults() {
+    return EncodingUtils.testBit(__isset_bitfield, __RETURNRESULTS_ISSET_ID);
+  }
+
+  public void setReturnResultsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RETURNRESULTS_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ROW:
@@ -413,6 +449,14 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
       }
       break;
 
+    case RETURN_RESULTS:
+      if (value == null) {
+        unsetReturnResults();
+      } else {
+        setReturnResults((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -433,6 +477,9 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
     case CELL_VISIBILITY:
       return getCellVisibility();
 
+    case RETURN_RESULTS:
+      return isReturnResults();
+
     }
     throw new IllegalStateException();
   }
@@ -454,6 +501,8 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
       return isSetDurability();
     case CELL_VISIBILITY:
       return isSetCellVisibility();
+    case RETURN_RESULTS:
+      return isSetReturnResults();
     }
     throw new IllegalStateException();
   }
@@ -516,6 +565,15 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
         return false;
     }
 
+    boolean this_present_returnResults = true && this.isSetReturnResults();
+    boolean that_present_returnResults = true && that.isSetReturnResults();
+    if (this_present_returnResults || that_present_returnResults) {
+      if (!(this_present_returnResults && that_present_returnResults))
+        return false;
+      if (this.returnResults != that.returnResults)
+        return false;
+    }
+
     return true;
   }
 
@@ -548,6 +606,11 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
     if (present_cellVisibility)
       list.add(cellVisibility);
 
+    boolean present_returnResults = true && (isSetReturnResults());
+    list.add(present_returnResults);
+    if (present_returnResults)
+      list.add(returnResults);
+
     return list.hashCode();
   }
 
@@ -609,6 +672,16 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetReturnResults()).compareTo(other.isSetReturnResults());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetReturnResults()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.returnResults, other.returnResults);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -674,6 +747,12 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
       }
       first = false;
     }
+    if (isSetReturnResults()) {
+      if (!first) sb.append(", ");
+      sb.append("returnResults:");
+      sb.append(this.returnResults);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -702,6 +781,8 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
 
   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
     try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
@@ -790,6 +871,14 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // RETURN_RESULTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.returnResults = iprot.readBool();
+              struct.setReturnResultsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -851,6 +940,11 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
           oprot.writeFieldEnd();
         }
       }
+      if (struct.isSetReturnResults()) {
+        oprot.writeFieldBegin(RETURN_RESULTS_FIELD_DESC);
+        oprot.writeBool(struct.returnResults);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -886,7 +980,10 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
       if (struct.isSetCellVisibility()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetReturnResults()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
       if (struct.isSetAttributes()) {
         {
           oprot.writeI32(struct.attributes.size());
@@ -903,6 +1000,9 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
       if (struct.isSetCellVisibility()) {
         struct.cellVisibility.write(oprot);
       }
+      if (struct.isSetReturnResults()) {
+        oprot.writeBool(struct.returnResults);
+      }
     }
 
     @Override
@@ -922,7 +1022,7 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
         }
       }
       struct.setColumnsIsSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TMap _map102 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
@@ -947,6 +1047,10 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
         struct.cellVisibility.read(iprot);
         struct.setCellVisibilityIsSet(true);
       }
+      if (incoming.get(3)) {
+        struct.returnResults = iprot.readBool();
+        struct.setReturnResultsIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
index 6c355a6..1b168d8 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
 


[46/47] hbase git commit: HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager

Posted by zh...@apache.org.
HBASE-21516 Use AsyncConnection instead of Connection in SecureBulkLoadManager


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

Branch: refs/heads/HBASE-21512
Commit: a13292db13ac8d09940fe8c1e9ec6b3e84b09271
Parents: f3caa01
Author: zhangduo <zh...@apache.org>
Authored: Sat Dec 1 21:15:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 31 20:34:24 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  5 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  7 ++-
 .../hbase/regionserver/HRegionServer.java       |  2 +-
 .../regionserver/SecureBulkLoadManager.java     | 24 +++++----
 .../hadoop/hbase/security/token/TokenUtil.java  | 57 +++++++++++++++-----
 .../hbase/security/token/TestTokenUtil.java     | 42 +++++++++++----
 6 files changed, 96 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a13292db/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index a3d49b5..d9e620b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -261,13 +261,12 @@ public final class ProtobufUtil {
    * just {@link ServiceException}. Prefer this method to
    * {@link #getRemoteException(ServiceException)} because trying to
    * contain direct protobuf references.
-   * @param e
    */
-  public static IOException handleRemoteException(Exception e) {
+  public static IOException handleRemoteException(Throwable e) {
     return makeIOExceptionOfException(e);
   }
 
-  private static IOException makeIOExceptionOfException(Exception e) {
+  private static IOException makeIOExceptionOfException(Throwable e) {
     Throwable t = e;
     if (e instanceof ServiceException ||
         e instanceof org.apache.hbase.thirdparty.com.google.protobuf.ServiceException) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13292db/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 fea81f1..de2fb7d 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
@@ -40,7 +40,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ByteBufferExtendedCell;
@@ -123,6 +122,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Service;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -343,13 +343,12 @@ public final class ProtobufUtil {
    * just {@link ServiceException}. Prefer this method to
    * {@link #getRemoteException(ServiceException)} because trying to
    * contain direct protobuf references.
-   * @param e
    */
-  public static IOException handleRemoteException(Exception e) {
+  public static IOException handleRemoteException(Throwable e) {
     return makeIOExceptionOfException(e);
   }
 
-  private static IOException makeIOExceptionOfException(Exception e) {
+  private static IOException makeIOExceptionOfException(Throwable e) {
     Throwable t = e;
     if (e instanceof ServiceException) {
       t = e.getCause();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13292db/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 375b3f8..6e5fc9f 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
@@ -1930,7 +1930,7 @@ public class HRegionServer extends HasThread implements
     if (!isStopped() && !isAborted()) {
       initializeThreads();
     }
-    this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, clusterConnection);
+    this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, asyncClusterConnection);
     this.secureBulkLoadManager.start();
 
     // Health checker thread.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13292db/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
index 566a6b6..add6519 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Consumer;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -38,11 +37,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +56,9 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
@@ -111,9 +113,9 @@ public class SecureBulkLoadManager {
 
   private UserProvider userProvider;
   private ConcurrentHashMap<UserGroupInformation, Integer> ugiReferenceCounter;
-  private Connection conn;
+  private AsyncConnection conn;
 
-  SecureBulkLoadManager(Configuration conf, Connection conn) {
+  SecureBulkLoadManager(Configuration conf, AsyncConnection conn) {
     this.conf = conf;
     this.conn = conn;
   }
@@ -212,23 +214,23 @@ public class SecureBulkLoadManager {
       familyPaths.add(new Pair<>(el.getFamily().toByteArray(), el.getPath()));
     }
 
-    Token userToken = null;
+    Token<AuthenticationTokenIdentifier> userToken = null;
     if (userProvider.isHadoopSecurityEnabled()) {
-      userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
-              .getPassword().toByteArray(), new Text(request.getFsToken().getKind()), new Text(
-              request.getFsToken().getService()));
+      userToken = new Token<>(request.getFsToken().getIdentifier().toByteArray(),
+        request.getFsToken().getPassword().toByteArray(), new Text(request.getFsToken().getKind()),
+        new Text(request.getFsToken().getService()));
     }
     final String bulkToken = request.getBulkToken();
     User user = getActiveUser();
     final UserGroupInformation ugi = user.getUGI();
     if (userProvider.isHadoopSecurityEnabled()) {
       try {
-        Token tok = TokenUtil.obtainToken(conn);
+        Token<AuthenticationTokenIdentifier> tok = TokenUtil.obtainToken(conn).get();
         if (tok != null) {
           boolean b = ugi.addToken(tok);
           LOG.debug("token added " + tok + " for user " + ugi + " return=" + b);
         }
-      } catch (IOException ioe) {
+      } catch (Exception ioe) {
         LOG.warn("unable to add token", ioe);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13292db/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
index c54d905..28efb84 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -15,27 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.security.token;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import java.util.concurrent.CompletableFuture;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.AuthenticationService;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenResponse;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
@@ -45,6 +47,8 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 /**
  * Utility methods for obtaining authentication tokens.
  */
@@ -64,12 +68,39 @@ public class TokenUtil {
 
   /**
    * Obtain and return an authentication token for the current user.
+   * @param conn The async HBase cluster connection
+   * @return the authentication token instance, wrapped by a {@link CompletableFuture}.
+   */
+  public static CompletableFuture<Token<AuthenticationTokenIdentifier>> obtainToken(
+      AsyncConnection conn) {
+    CompletableFuture<Token<AuthenticationTokenIdentifier>> future = new CompletableFuture<>();
+    if (injectedException != null) {
+      future.completeExceptionally(injectedException);
+      return future;
+    }
+    AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
+    table.<AuthenticationService.Interface, GetAuthenticationTokenResponse> coprocessorService(
+      AuthenticationProtos.AuthenticationService::newStub,
+      (s, c, r) -> s.getAuthenticationToken(c,
+        AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance(), r),
+      HConstants.EMPTY_START_ROW).whenComplete((resp, error) -> {
+        if (error != null) {
+          future.completeExceptionally(ProtobufUtil.handleRemoteException(error));
+        } else {
+          future.complete(toToken(resp.getToken()));
+        }
+      });
+    return future;
+  }
+
+  /**
+   * Obtain and return an authentication token for the current user.
    * @param conn The HBase cluster connection
    * @throws IOException if a remote error or serialization problem occurs.
    * @return the authentication token instance
    */
-  public static Token<AuthenticationTokenIdentifier> obtainToken(
-      Connection conn) throws IOException {
+  public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn)
+      throws IOException {
     Table meta = null;
     try {
       injectFault();
@@ -77,9 +108,9 @@ public class TokenUtil {
       meta = conn.getTable(TableName.META_TABLE_NAME);
       CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);
       AuthenticationProtos.AuthenticationService.BlockingInterface service =
-          AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
-      AuthenticationProtos.GetAuthenticationTokenResponse response = service.getAuthenticationToken(null,
-          AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance());
+        AuthenticationService.newBlockingStub(rpcChannel);
+      GetAuthenticationTokenResponse response =
+        service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance());
 
       return toToken(response.getToken());
     } catch (ServiceException se) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a13292db/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
index 32fcddb..585a3ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenUtil.java
@@ -18,35 +18,53 @@
 package org.apache.hadoop.hbase.security.token;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.net.URL;
 import java.net.URLClassLoader;
-
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 @Category(SmallTests.class)
 public class TestTokenUtil {
+
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestTokenUtil.class);
+    HBaseClassTestRule.forClass(TestTokenUtil.class);
 
-  @Test
-  public void testObtainToken() throws Exception {
+  private URLClassLoader cl;
+
+  @Before
+  public void setUp() {
     URL urlPU = ProtobufUtil.class.getProtectionDomain().getCodeSource().getLocation();
     URL urlTU = TokenUtil.class.getProtectionDomain().getCodeSource().getLocation();
+    cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+  }
 
-    ClassLoader cl = new URLClassLoader(new URL[] { urlPU, urlTU }, getClass().getClassLoader());
+  @After
+  public void tearDown() throws IOException {
+    Closeables.close(cl, true);
+  }
 
+  @Test
+  public void testObtainToken() throws Exception {
     Throwable injected = new com.google.protobuf.ServiceException("injected");
 
     Class<?> tokenUtil = cl.loadClass(TokenUtil.class.getCanonicalName());
@@ -55,8 +73,7 @@ public class TestTokenUtil {
     shouldInjectFault.set(null, injected);
 
     try {
-      tokenUtil.getMethod("obtainToken", Connection.class)
-          .invoke(null, new Object[] { null });
+      tokenUtil.getMethod("obtainToken", Connection.class).invoke(null, new Object[] { null });
       fail("Should have injected exception.");
     } catch (InvocationTargetException e) {
       Throwable t = e;
@@ -72,9 +89,16 @@ public class TestTokenUtil {
       }
     }
 
+    CompletableFuture<?> future = (CompletableFuture<?>) tokenUtil
+      .getMethod("obtainToken", AsyncConnection.class).invoke(null, new Object[] { null });
+    try {
+      future.get();
+      fail("Should have injected exception.");
+    } catch (ExecutionException e) {
+      assertSame(injected, e.getCause());
+    }
     Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
-        .getDeclaredMethod("isClassLoaderLoaded")
-        .invoke(null);
+      .getDeclaredMethod("isClassLoaderLoaded").invoke(null);
     assertFalse("Should not have loaded DynamicClassLoader", loaded);
   }
 }


[06/47] hbase git commit: HBASE-21590 Optimize trySkipToNextColumn in StoreScanner a bit.

Posted by zh...@apache.org.
HBASE-21590 Optimize trySkipToNextColumn in StoreScanner a bit.


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

Branch: refs/heads/HBASE-21512
Commit: cb1966dc2d94fba10d9b6af3c5719e03f621df92
Parents: f32d261
Author: Lars Hofhansl <la...@apache.org>
Authored: Thu Dec 13 11:57:16 2018 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Thu Dec 13 11:57:16 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/StoreScanner.java  | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cb1966dc/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 736c08a..e7a4528 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
@@ -802,12 +802,16 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   @VisibleForTesting
   protected boolean trySkipToNextRow(Cell cell) throws IOException {
     Cell nextCell = null;
+    // used to guard against a changed next indexed key by doing a identity comparison
+    // when the identity changes we need to compare the bytes again
+    Cell previousIndexedKey = null;
     do {
       Cell nextIndexedKey = getNextIndexedKey();
       if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0) {
+          && (nextIndexedKey == previousIndexedKey || matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0)) {
         this.heap.next();
         ++kvsScanned;
+        previousIndexedKey = nextIndexedKey;
       } else {
         return false;
       }
@@ -823,12 +827,16 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   @VisibleForTesting
   protected boolean trySkipToNextColumn(Cell cell) throws IOException {
     Cell nextCell = null;
+    // used to guard against a changed next indexed key by doing a identity comparison
+    // when the identity changes we need to compare the bytes again
+    Cell previousIndexedKey = null;
     do {
       Cell nextIndexedKey = getNextIndexedKey();
       if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0) {
+          && (nextIndexedKey == previousIndexedKey || matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0)) {
         this.heap.next();
         ++kvsScanned;
+        previousIndexedKey = nextIndexedKey;
       } else {
         return false;
       }


[22/47] hbase git commit: HBASE-21618 Scan with the same startRow(inclusive=true) and stopRow(inclusive=false) returns one result

Posted by zh...@apache.org.
HBASE-21618 Scan with the same startRow(inclusive=true) and stopRow(inclusive=false) returns one result


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

Branch: refs/heads/HBASE-21512
Commit: ad819380c744678e719431fb8b1b5e1951bc31b6
Parents: 7875673
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Dec 20 11:03:54 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Dec 21 09:49:24 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  4 +--
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  4 +--
 .../hbase/shaded/protobuf/TestProtobufUtil.java |  1 +
 .../hbase/client/TestFromClientSide3.java       |  4 +--
 .../client/TestScannersFromClientSide.java      | 38 ++++++++++++++++++++
 .../hadoop/hbase/protobuf/TestProtobufUtil.java |  1 +
 6 files changed, 44 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ad819380/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 4d54528..a3d49b5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -937,9 +937,7 @@ public final class ProtobufUtil {
     if (!scan.includeStartRow()) {
       scanBuilder.setIncludeStartRow(false);
     }
-    if (scan.includeStopRow()) {
-      scanBuilder.setIncludeStopRow(true);
-    }
+    scanBuilder.setIncludeStopRow(scan.includeStopRow());
     if (scan.getReadType() != Scan.ReadType.DEFAULT) {
       scanBuilder.setReadType(toReadType(scan.getReadType()));
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad819380/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 cf4c831..fea81f1 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
@@ -1081,9 +1081,7 @@ public final class ProtobufUtil {
     if (!scan.includeStartRow()) {
       scanBuilder.setIncludeStartRow(false);
     }
-    if (scan.includeStopRow()) {
-      scanBuilder.setIncludeStopRow(true);
-    }
+    scanBuilder.setIncludeStopRow(scan.includeStopRow());
     if (scan.getReadType() != Scan.ReadType.DEFAULT) {
       scanBuilder.setReadType(toReadType(scan.getReadType()));
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad819380/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
index be51e96..2d8a74a 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/shaded/protobuf/TestProtobufUtil.java
@@ -246,6 +246,7 @@ public class TestProtobufUtil {
     scanBuilder.setCacheBlocks(false);
     scanBuilder.setCaching(1024);
     scanBuilder.setTimeRange(ProtobufUtil.toTimeRange(TimeRange.allTime()));
+    scanBuilder.setIncludeStopRow(false);
     ClientProtos.Scan expectedProto = scanBuilder.build();
 
     ClientProtos.Scan actualProto = ProtobufUtil.toScan(

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad819380/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 0dee20b..cbfa1bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -1094,7 +1094,7 @@ public class TestFromClientSide3 {
     }
 
     Scan scan = new Scan();
-    scan.withStartRow(ROW).withStopRow(ROW).addFamily(FAMILY).setBatch(3)
+    scan.withStartRow(ROW).withStopRow(ROW, true).addFamily(FAMILY).setBatch(3)
         .setMaxResultSize(4 * 1024 * 1024);
     Result result;
     try (ResultScanner scanner = table.getScanner(scan)) {
@@ -1117,7 +1117,7 @@ public class TestFromClientSide3 {
     }
 
     scan = new Scan();
-    scan.withStartRow(ROW).withStopRow(ROW).addFamily(FAMILY).setBatch(2)
+    scan.withStartRow(ROW).withStopRow(ROW, true).addFamily(FAMILY).setBatch(2)
         .setMaxResultSize(4 * 1024 * 1024);
     try (ResultScanner scanner = table.getScanner(scan)) {
       List<Result> list = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad819380/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 5e8d107..b91e205 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
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.ipc.RpcClient.DEFAULT_CODEC_CLASS;
 import static org.junit.Assert.assertArrayEquals;
 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;
@@ -897,4 +898,41 @@ public class TestScannersFromClientSide {
       }
     }
   }
+
+  @Test
+  public void testScanWithSameStartRowStopRow() throws IOException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    try (Table table = TEST_UTIL.createTable(tableName, FAMILY)) {
+      table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE));
+
+      Scan scan = new Scan().withStartRow(ROW).withStopRow(ROW);
+      try (ResultScanner scanner = table.getScanner(scan)) {
+        assertNull(scanner.next());
+      }
+
+      scan = new Scan().withStartRow(ROW, true).withStopRow(ROW, true);
+      try (ResultScanner scanner = table.getScanner(scan)) {
+        Result result = scanner.next();
+        assertNotNull(result);
+        assertArrayEquals(ROW, result.getRow());
+        assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER));
+        assertNull(scanner.next());
+      }
+
+      scan = new Scan().withStartRow(ROW, true).withStopRow(ROW, false);
+      try (ResultScanner scanner = table.getScanner(scan)) {
+        assertNull(scanner.next());
+      }
+
+      scan = new Scan().withStartRow(ROW, false).withStopRow(ROW, false);
+      try (ResultScanner scanner = table.getScanner(scan)) {
+        assertNull(scanner.next());
+      }
+
+      scan = new Scan().withStartRow(ROW, false).withStopRow(ROW, true);
+      try (ResultScanner scanner = table.getScanner(scan)) {
+        assertNull(scanner.next());
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad819380/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 300f029..ff29df8 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
@@ -319,6 +319,7 @@ public class TestProtobufUtil {
     scanBuilder.setCacheBlocks(false);
     scanBuilder.setCaching(1024);
     scanBuilder.setTimeRange(ProtobufUtil.toTimeRange(TimeRange.allTime()));
+    scanBuilder.setIncludeStopRow(false);
     ClientProtos.Scan expectedProto = scanBuilder.build();
 
     ClientProtos.Scan actualProto = ProtobufUtil.toScan(


[05/47] hbase git commit: HBASE-21582 If call HBaseAdmin#snapshotAsync but forget call isSnapshotFinished, then SnapshotHFileCleaner will skip to run every time

Posted by zh...@apache.org.
HBASE-21582 If call HBaseAdmin#snapshotAsync but forget call isSnapshotFinished, then SnapshotHFileCleaner will skip to run every time


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

Branch: refs/heads/HBASE-21512
Commit: f32d2618430f70e1b0db92785294b2c7892cc02b
Parents: 4640ff5
Author: huzheng <op...@gmail.com>
Authored: Tue Dec 11 20:27:56 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Thu Dec 13 10:35:20 2018 +0800

----------------------------------------------------------------------
 .../hbase/master/snapshot/SnapshotManager.java  | 48 ++++++++++++++------
 .../master/cleaner/TestSnapshotFromMaster.java  | 27 ++++++++++-
 .../master/snapshot/TestSnapshotManager.java    | 36 +++++++++++++--
 3 files changed, 92 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f32d2618/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index 2b963b2..05db4ab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -28,7 +28,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -91,6 +95,8 @@ 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.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Type;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * This class manages the procedure of taking and restoring snapshots. There is only one
@@ -120,7 +126,9 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * At this point, if the user asks for the snapshot/restore status, the result will be
    * snapshot done if exists or failed if it doesn't exists.
    */
-  private static final int SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT = 60 * 1000;
+  public static final String HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS =
+      "hbase.snapshot.sentinels.cleanup.timeoutMillis";
+  public static final long SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT = 60 * 1000L;
 
   /** Enable or disable snapshot support */
   public static final String HBASE_SNAPSHOT_ENABLED = "hbase.snapshot.enabled";
@@ -151,7 +159,11 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
   // The map is always accessed and modified under the object lock using synchronized.
   // snapshotTable() will insert an Handler in the table.
   // isSnapshotDone() will remove the handler requested if the operation is finished.
-  private Map<TableName, SnapshotSentinel> snapshotHandlers = new ConcurrentHashMap<>();
+  private final Map<TableName, SnapshotSentinel> snapshotHandlers = new ConcurrentHashMap<>();
+  private final ScheduledExecutorService scheduleThreadPool =
+      Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
+          .setNameFormat("SnapshotHandlerChoreCleaner").setDaemon(true).build());
+  private ScheduledFuture<?> snapshotHandlerChoreCleanerTask;
 
   // Restore map, with table name as key, procedure ID as value.
   // The map is always accessed and modified under the object lock using synchronized.
@@ -181,17 +193,21 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @param coordinator procedure coordinator instance.  exposed for testing.
    * @param pool HBase ExecutorServcie instance, exposed for testing.
    */
-  public SnapshotManager(final MasterServices master, final MetricsMaster metricsMaster,
-      ProcedureCoordinator coordinator, ExecutorService pool)
+  @VisibleForTesting
+  SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator,
+      ExecutorService pool, int sentinelCleanInterval)
       throws IOException, UnsupportedOperationException {
     this.master = master;
 
     this.rootDir = master.getMasterFileSystem().getRootDir();
-    checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
+    Configuration conf = master.getConfiguration();
+    checkSnapshotSupport(conf, master.getMasterFileSystem());
 
     this.coordinator = coordinator;
     this.executorService = pool;
     resetTempDir();
+    snapshotHandlerChoreCleanerTask = this.scheduleThreadPool.scheduleAtFixedRate(
+      this::cleanupSentinels, sentinelCleanInterval, sentinelCleanInterval, TimeUnit.SECONDS);
   }
 
   /**
@@ -274,7 +290,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    *
    * @throws IOException if we can't reach the filesystem
    */
-  void resetTempDir() throws IOException {
+  private void resetTempDir() throws IOException {
     // cleanup any existing snapshots.
     Path tmpdir = SnapshotDescriptionUtils.getWorkingSnapshotDir(rootDir,
         master.getConfiguration());
@@ -290,7 +306,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @throws SnapshotDoesNotExistException If the specified snapshot does not exist.
    * @throws IOException For filesystem IOExceptions
    */
-  public void deleteSnapshot(SnapshotDescription snapshot) throws SnapshotDoesNotExistException, IOException {
+  public void deleteSnapshot(SnapshotDescription snapshot) throws IOException {
     // check to see if it is completed
     if (!isSnapshotCompleted(snapshot)) {
       throw new SnapshotDoesNotExistException(ProtobufUtil.createSnapshotDesc(snapshot));
@@ -934,7 +950,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
       this.restoreTableToProcIdMap.remove(tableName);
       return false;
     }
-
   }
 
   /**
@@ -989,14 +1004,15 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    */
   private synchronized void cleanupSentinels(final Map<TableName, SnapshotSentinel> sentinels) {
     long currentTime = EnvironmentEdgeManager.currentTime();
-    Iterator<Map.Entry<TableName, SnapshotSentinel>> it =
-        sentinels.entrySet().iterator();
+    long sentinelsCleanupTimeoutMillis =
+        master.getConfiguration().getLong(HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS,
+          SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT);
+    Iterator<Map.Entry<TableName, SnapshotSentinel>> it = sentinels.entrySet().iterator();
     while (it.hasNext()) {
       Map.Entry<TableName, SnapshotSentinel> entry = it.next();
       SnapshotSentinel sentinel = entry.getValue();
-      if (sentinel.isFinished() &&
-          (currentTime - sentinel.getCompletionTimestamp()) > SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT)
-      {
+      if (sentinel.isFinished()
+          && (currentTime - sentinel.getCompletionTimestamp()) > sentinelsCleanupTimeoutMillis) {
         it.remove();
       }
     }
@@ -1031,7 +1047,9 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     for (SnapshotSentinel snapshotHandler: this.snapshotHandlers.values()) {
       snapshotHandler.cancel(why);
     }
-
+    if (snapshotHandlerChoreCleanerTask != null) {
+      snapshotHandlerChoreCleanerTask.cancel(true);
+    }
     try {
       if (coordinator != null) {
         coordinator.close();
@@ -1166,6 +1184,8 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency);
     this.executorService = master.getExecutorService();
     resetTempDir();
+    snapshotHandlerChoreCleanerTask =
+        scheduleThreadPool.scheduleAtFixedRate(this::cleanupSentinels, 10, 10, TimeUnit.SECONDS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f32d2618/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index 9d76ede..cc2ee06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.master.cleaner;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -25,6 +26,8 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
+import java.util.regex.Pattern;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,8 +35,12 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.SnapshotType;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -129,11 +136,11 @@ public class TestSnapshotFromMaster {
     conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, "");
     // Enable snapshot
     conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
+    conf.setLong(SnapshotManager.HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS, 3 * 1000L);
     conf.setLong(SnapshotHFileCleaner.HFILE_CACHE_REFRESH_PERIOD_CONF_KEY, cacheRefreshPeriod);
     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
       ConstantSizeRegionSplitPolicy.class.getName());
     conf.setInt("hbase.hfile.compactions.cleaner.interval", 20 * 1000);
-
   }
 
   @Before
@@ -419,4 +426,22 @@ public class TestSnapshotFromMaster {
     builder.commit();
     return builder.getSnapshotDescription();
   }
+
+  @Test
+  public void testAsyncSnapshotWillNotBlockSnapshotHFileCleaner() throws Exception {
+    // Write some data
+    Table table = UTIL.getConnection().getTable(TABLE_NAME);
+    for (int i = 0; i < 10; i++) {
+      Put put = new Put(Bytes.toBytes(i)).addColumn(TEST_FAM, Bytes.toBytes("q"), Bytes.toBytes(i));
+      table.put(put);
+    }
+    String snapshotName = "testAsyncSnapshotWillNotBlockSnapshotHFileCleaner01";
+    UTIL.getAdmin().snapshotAsync(new org.apache.hadoop.hbase.client.SnapshotDescription(
+        snapshotName, TABLE_NAME, SnapshotType.FLUSH));
+    Waiter.waitFor(UTIL.getConfiguration(), 10 * 1000L, 200L,
+      () -> UTIL.getAdmin().listSnapshots(Pattern.compile(snapshotName)).size() == 1);
+    assertTrue(master.getSnapshotManager().isTakingAnySnapshot());
+    Thread.sleep(11 * 1000L);
+    assertFalse(master.getSnapshotManager().isTakingAnySnapshot());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f32d2618/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
index 3a6a61f..ff903c4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.MetricsMaster;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
@@ -62,7 +61,6 @@ public class TestSnapshotManager {
   public TestName name = new TestName();
 
   MasterServices services = Mockito.mock(MasterServices.class);
-  MetricsMaster metrics = Mockito.mock(MetricsMaster.class);
   ProcedureCoordinator coordinator = Mockito.mock(ProcedureCoordinator.class);
   ExecutorService pool = Mockito.mock(ExecutorService.class);
   MasterFileSystem mfs = Mockito.mock(MasterFileSystem.class);
@@ -79,14 +77,44 @@ public class TestSnapshotManager {
     return getNewManager(UTIL.getConfiguration());
   }
 
-  private SnapshotManager getNewManager(final Configuration conf)
+  private SnapshotManager getNewManager(Configuration conf) throws IOException, KeeperException {
+    return getNewManager(conf, 1);
+  }
+
+  private SnapshotManager getNewManager(Configuration conf, int intervalSeconds)
       throws IOException, KeeperException {
     Mockito.reset(services);
     Mockito.when(services.getConfiguration()).thenReturn(conf);
     Mockito.when(services.getMasterFileSystem()).thenReturn(mfs);
     Mockito.when(mfs.getFileSystem()).thenReturn(fs);
     Mockito.when(mfs.getRootDir()).thenReturn(UTIL.getDataTestDir());
-    return new SnapshotManager(services, metrics, coordinator, pool);
+    return new SnapshotManager(services, coordinator, pool, intervalSeconds);
+  }
+
+  @Test
+  public void testCleanFinishedHandler() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    Configuration conf = UTIL.getConfiguration();
+    try {
+      conf.setLong(SnapshotManager.HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS, 5 * 1000L);
+      SnapshotManager manager = getNewManager(conf, 1);
+      TakeSnapshotHandler handler = Mockito.mock(TakeSnapshotHandler.class);
+      assertFalse("Manager is in process when there is no current handler",
+        manager.isTakingSnapshot(tableName));
+      manager.setSnapshotHandlerForTesting(tableName, handler);
+      Mockito.when(handler.isFinished()).thenReturn(false);
+      assertTrue(manager.isTakingAnySnapshot());
+      assertTrue("Manager isn't in process when handler is running",
+        manager.isTakingSnapshot(tableName));
+      Mockito.when(handler.isFinished()).thenReturn(true);
+      assertFalse("Manager is process when handler isn't running",
+        manager.isTakingSnapshot(tableName));
+      assertTrue(manager.isTakingAnySnapshot());
+      Thread.sleep(6 * 1000);
+      assertFalse(manager.isTakingAnySnapshot());
+    } finally {
+      conf.unset(SnapshotManager.HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS);
+    }
   }
 
   @Test


[17/47] hbase git commit: HBASE-21592 quota.addGetResult(r) throw NPE

Posted by zh...@apache.org.
HBASE-21592 quota.addGetResult(r) throw NPE

Signed-off-by: huzheng <op...@gmail.com>


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

Branch: refs/heads/HBASE-21512
Commit: f78284685fc533230a0395d297ebacff32632396
Parents: 1971d02
Author: xuqinya <xu...@163.com>
Authored: Tue Dec 18 08:19:47 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Tue Dec 18 16:15:51 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/RSRpcServices.java   |  3 ++-
 .../hadoop/hbase/quotas/TestQuotaThrottle.java     | 17 +++++++++++++++++
 2 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f7828468/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 31df37a..f788a86 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -2571,7 +2571,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         }
         builder.setResult(pbr);
       }
-      if (r != null) {
+      //r.cells is null when an table.exists(get) call
+      if (r != null && r.rawCells() != null) {
         quota.addGetResult(r);
       }
       return builder.build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f7828468/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
index e506a08..c069403 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
@@ -553,6 +553,23 @@ public class TestQuotaThrottle {
     triggerTableCacheRefresh(true, TABLE_NAMES[0]);
   }
 
+  @Test
+  public void testTableExistsGetThrottle() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+
+    // Add throttle quota
+    admin.setQuota(QuotaSettingsFactory.throttleTable(TABLE_NAMES[0],
+        ThrottleType.REQUEST_NUMBER, 100, TimeUnit.MINUTES));
+    triggerTableCacheRefresh(false, TABLE_NAMES[0]);
+
+    Table table = TEST_UTIL.getConnection().getTable(TABLE_NAMES[0]);
+    // An exists call when having throttle quota
+    table.exists(new Get(Bytes.toBytes("abc")));
+
+    admin.setQuota(QuotaSettingsFactory.unthrottleTable(TABLE_NAMES[0]));
+    triggerTableCacheRefresh(true, TABLE_NAMES[0]);
+  }
+
   private int doPuts(int maxOps, final Table... tables) throws Exception {
     return doPuts(maxOps, -1, tables);
   }


[25/47] hbase git commit: HBASE-21631: list_quotas should print human readable values for LIMIT

Posted by zh...@apache.org.
HBASE-21631: list_quotas should print human readable values for LIMIT

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

Branch: refs/heads/HBASE-21512
Commit: b2bf22e209d2e87121986b35c5749b2b8ae45fa2
Parents: e160b5a
Author: Sakthi <sa...@gmail.com>
Authored: Fri Dec 21 16:23:08 2018 -0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Sat Dec 22 22:00:58 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java    | 2 +-
 .../org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b2bf22e2/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
index 02bd6e4..8b31e94 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/SpaceLimitSettings.java
@@ -205,7 +205,7 @@ class SpaceLimitSettings extends QuotaSettings {
     if (proto.getQuota().getRemove()) {
       sb.append(", REMOVE => ").append(proto.getQuota().getRemove());
     } else {
-      sb.append(", LIMIT => ").append(proto.getQuota().getSoftLimit());
+      sb.append(", LIMIT => ").append(sizeToString(proto.getQuota().getSoftLimit()));
       sb.append(", VIOLATION_POLICY => ").append(proto.getQuota().getViolationPolicy());
     }
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b2bf22e2/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
index 0c6cb81..e47e4ff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
@@ -276,7 +276,7 @@ public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
       if (spaceProto.getRemove()) {
         builder.append(", REMOVE => ").append(spaceProto.getRemove());
       } else {
-        builder.append(", LIMIT => ").append(spaceProto.getSoftLimit());
+        builder.append(", LIMIT => ").append(sizeToString(spaceProto.getSoftLimit()));
         builder.append(", VIOLATION_POLICY => ").append(spaceProto.getViolationPolicy());
       }
       builder.append(" } ");


[07/47] hbase git commit: Revert "HIVE-21575 : memstore above high watermark message is logged too much"

Posted by zh...@apache.org.
Revert "HIVE-21575 : memstore above high watermark message is logged too much"

This reverts commit 4640ff5959af4865966126a503a7cd15e26a7408.


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

Branch: refs/heads/HBASE-21512
Commit: 9a25d0c249e595a1f8aef41cd677b44ff1c72d73
Parents: cb1966d
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Dec 13 12:46:39 2018 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Dec 13 12:46:39 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/9a25d0c2/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 804a2f8..699c9b6 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
@@ -703,7 +703,6 @@ class MemStoreFlusher implements FlushRequester {
     if (flushType != FlushType.NORMAL) {
       TraceUtil.addTimelineAnnotation("Force Flush. We're above high water mark.");
       long start = EnvironmentEdgeManager.currentTime();
-      long nextLogTimeMs = start;
       synchronized (this.blockSignal) {
         boolean blocked = false;
         long startTime = 0;
@@ -745,11 +744,8 @@ class MemStoreFlusher implements FlushRequester {
               LOG.warn("Interrupted while waiting");
               interrupted = true;
             }
-            long nowMs = EnvironmentEdgeManager.currentTime();
-            if (nowMs >= nextLogTimeMs) {
-              LOG.warn("Memstore is above high water mark and block {} ms", nowMs - start);
-              nextLogTimeMs = nowMs + 1000;
-            }
+            long took = EnvironmentEdgeManager.currentTime() - start;
+            LOG.warn("Memstore is above high water mark and block " + took + "ms");
             flushType = isAboveHighWaterMark();
           }
         } finally {


[08/47] hbase git commit: HBASE-21575 : memstore above high watermark message is logged too much

Posted by zh...@apache.org.
HBASE-21575 : memstore above high watermark message is logged too much


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

Branch: refs/heads/HBASE-21512
Commit: 3ff274e22eb5710f4301fb0fce364e22a11288d7
Parents: 9a25d0c
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Dec 12 11:02:25 2018 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Dec 13 12:47:11 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/3ff274e2/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 699c9b6..804a2f8 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
@@ -703,6 +703,7 @@ class MemStoreFlusher implements FlushRequester {
     if (flushType != FlushType.NORMAL) {
       TraceUtil.addTimelineAnnotation("Force Flush. We're above high water mark.");
       long start = EnvironmentEdgeManager.currentTime();
+      long nextLogTimeMs = start;
       synchronized (this.blockSignal) {
         boolean blocked = false;
         long startTime = 0;
@@ -744,8 +745,11 @@ class MemStoreFlusher implements FlushRequester {
               LOG.warn("Interrupted while waiting");
               interrupted = true;
             }
-            long took = EnvironmentEdgeManager.currentTime() - start;
-            LOG.warn("Memstore is above high water mark and block " + took + "ms");
+            long nowMs = EnvironmentEdgeManager.currentTime();
+            if (nowMs >= nextLogTimeMs) {
+              LOG.warn("Memstore is above high water mark and block {} ms", nowMs - start);
+              nextLogTimeMs = nowMs + 1000;
+            }
             flushType = isAboveHighWaterMark();
           }
         } finally {


[04/47] hbase git commit: HIVE-21575 : memstore above high watermark message is logged too much

Posted by zh...@apache.org.
HIVE-21575 : memstore above high watermark message is logged too much


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

Branch: refs/heads/HBASE-21512
Commit: 4640ff5959af4865966126a503a7cd15e26a7408
Parents: 67d6d50
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Dec 12 11:02:25 2018 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Dec 12 11:02:25 2018 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/4640ff59/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 699c9b6..804a2f8 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
@@ -703,6 +703,7 @@ class MemStoreFlusher implements FlushRequester {
     if (flushType != FlushType.NORMAL) {
       TraceUtil.addTimelineAnnotation("Force Flush. We're above high water mark.");
       long start = EnvironmentEdgeManager.currentTime();
+      long nextLogTimeMs = start;
       synchronized (this.blockSignal) {
         boolean blocked = false;
         long startTime = 0;
@@ -744,8 +745,11 @@ class MemStoreFlusher implements FlushRequester {
               LOG.warn("Interrupted while waiting");
               interrupted = true;
             }
-            long took = EnvironmentEdgeManager.currentTime() - start;
-            LOG.warn("Memstore is above high water mark and block " + took + "ms");
+            long nowMs = EnvironmentEdgeManager.currentTime();
+            if (nowMs >= nextLogTimeMs) {
+              LOG.warn("Memstore is above high water mark and block {} ms", nowMs - start);
+              nextLogTimeMs = nowMs + 1000;
+            }
             flushType = isAboveHighWaterMark();
           }
         } finally {


[45/47] hbase git commit: HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer

Posted by zh...@apache.org.
HBASE-21515 Also initialize an AsyncClusterConnection in HRegionServer


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

Branch: refs/heads/HBASE-21512
Commit: f3caa0188b5b5f032d4b90a7b75e518a8752e0f4
Parents: 7755d4b
Author: zhangduo <zh...@apache.org>
Authored: Fri Nov 30 08:23:47 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 31 20:34:24 2018 +0800

----------------------------------------------------------------------
 .../hbase/client/AsyncClusterConnection.java    | 38 ++++++++++++
 .../hbase/client/AsyncConnectionImpl.java       | 39 ++++++------
 .../hbase/client/ClusterConnectionFactory.java  | 63 ++++++++++++++++++++
 .../hadoop/hbase/client/ConnectionFactory.java  |  5 +-
 .../hadoop/hbase/util/ReflectionUtils.java      | 22 ++++---
 .../java/org/apache/hadoop/hbase/Server.java    | 20 +++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |  3 +
 .../hbase/regionserver/HRegionServer.java       | 56 ++++++++++++-----
 .../regionserver/ReplicationSyncUp.java         |  6 ++
 .../hadoop/hbase/MockRegionServerServices.java  |  5 ++
 .../client/TestAsyncNonMetaRegionLocator.java   |  2 +-
 ...syncNonMetaRegionLocatorConcurrenyLimit.java |  2 +-
 .../client/TestAsyncRegionLocatorTimeout.java   |  2 +-
 ...TestAsyncSingleRequestRpcRetryingCaller.java |  4 +-
 .../hbase/client/TestAsyncTableNoncedRetry.java |  2 +-
 .../hbase/master/MockNoopMasterServices.java    |  6 ++
 .../hadoop/hbase/master/MockRegionServer.java   |  5 ++
 .../hbase/master/TestActiveMasterManager.java   |  6 ++
 .../hbase/master/cleaner/TestHFileCleaner.java  |  6 ++
 .../master/cleaner/TestHFileLinkCleaner.java    |  6 ++
 .../hbase/master/cleaner/TestLogsCleaner.java   |  6 ++
 .../cleaner/TestReplicationHFileCleaner.java    |  6 ++
 .../regionserver/TestHeapMemoryManager.java     |  6 ++
 .../hbase/regionserver/TestSplitLogWorker.java  |  6 ++
 .../hbase/regionserver/TestWALLockup.java       |  6 ++
 .../TestReplicationTrackerZKImpl.java           |  6 ++
 .../TestReplicationSourceManager.java           |  6 ++
 .../security/token/TestTokenAuthentication.java |  6 ++
 .../apache/hadoop/hbase/util/MockServer.java    |  6 ++
 29 files changed, 302 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
new file mode 100644
index 0000000..c7dea25
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -0,0 +1,38 @@
+/**
+ * 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.ipc.RpcClient;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The asynchronous connection for internal usage.
+ */
+@InterfaceAudience.Private
+public interface AsyncClusterConnection extends AsyncConnection {
+
+  /**
+   * Get the nonce generator for this connection.
+   */
+  NonceGenerator getNonceGenerator();
+
+  /**
+   * Get the rpc client we used to communicate with other servers.
+   */
+  RpcClient getRpcClient();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 078395b..79ec54b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -21,48 +21,48 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.NO_NONCE_GENERATOR;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.getStubKey;
 import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLED_KEY;
 
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
+
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.util.CollectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
 
 /**
  * The implementation of AsyncConnection.
  */
 @InterfaceAudience.Private
-class AsyncConnectionImpl implements AsyncConnection {
+class AsyncConnectionImpl implements AsyncClusterConnection {
 
   private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);
 
@@ -105,7 +105,7 @@ class AsyncConnectionImpl implements AsyncConnection {
   private ChoreService authService;
 
   public AsyncConnectionImpl(Configuration conf, AsyncRegistry registry, String clusterId,
-      User user) {
+      SocketAddress localAddress, User user) {
     this.conf = conf;
     this.user = user;
     if (user.isLoginFromKeytab()) {
@@ -113,7 +113,7 @@ class AsyncConnectionImpl implements AsyncConnection {
     }
     this.connConf = new AsyncConnectionConfiguration(conf);
     this.registry = registry;
-    this.rpcClient = RpcClientFactory.createClient(conf, clusterId);
+    this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, null);
     this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
     this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true);
     this.rpcTimeout =
@@ -158,11 +158,16 @@ class AsyncConnectionImpl implements AsyncConnection {
   }
 
   // ditto
-  @VisibleForTesting
+  @Override
   public NonceGenerator getNonceGenerator() {
     return nonceGenerator;
   }
 
+  @Override
+  public RpcClient getRpcClient() {
+    return rpcClient;
+  }
+
   private ClientService.Interface createRegionServerStub(ServerName serverName) throws IOException {
     return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
new file mode 100644
index 0000000..68c0630
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java
@@ -0,0 +1,63 @@
+/**
+ * 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.io.InterruptedIOException;
+import java.net.SocketAddress;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+/**
+ * The factory for creating {@link AsyncClusterConnection}.
+ */
+@InterfaceAudience.Private
+public final class ClusterConnectionFactory {
+
+  private ClusterConnectionFactory() {
+  }
+
+  /**
+   * Create a new {@link AsyncClusterConnection} instance.
+   * <p/>
+   * Unlike what we have done in {@link ConnectionFactory}, here we just return an
+   * {@link AsyncClusterConnection} instead of a {@link java.util.concurrent.CompletableFuture},
+   * which means this method could block on fetching the cluster id. This is just used to simplify
+   * the implementation, as when starting new region servers, we do not need to be event-driven. Can
+   * change later if we want a {@link java.util.concurrent.CompletableFuture} here.
+   */
+  public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf,
+      SocketAddress localAddress, User user) throws IOException {
+    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(conf);
+    String clusterId;
+    try {
+      clusterId = registry.getClusterId().get();
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException().initCause(e);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      Throwables.propagateIfPossible(cause, IOException.class);
+      throw new IOException(cause);
+    }
+    return new AsyncConnectionImpl(conf, registry, clusterId, localAddress, user);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index e24af74..2ba732a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -295,9 +295,8 @@ public class ConnectionFactory {
         AsyncConnectionImpl.class, AsyncConnection.class);
       try {
         future.complete(
-          user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>)() ->
-            ReflectionUtils.newInstance(clazz, conf, registry, clusterId, user))
-        );
+          user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>) () -> ReflectionUtils
+            .newInstance(clazz, conf, registry, clusterId, null, user)));
       } catch (Exception e) {
         future.completeExceptionally(e);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
index a136846..268249d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
@@ -83,15 +83,19 @@ public class ReflectionUtils {
 
       boolean match = true;
       for (int i = 0; i < ctorParamTypes.length && match; ++i) {
-        Class<?> paramType = paramTypes[i].getClass();
-        match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType) :
-                  ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
-                   (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
-                   (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
-                   (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
-                   (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
-                   (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
-                   (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+        if (paramTypes[i] == null) {
+          match = !ctorParamTypes[i].isPrimitive();
+        } else {
+          Class<?> paramType = paramTypes[i].getClass();
+          match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType)
+            : ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
+              (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
+              (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
+              (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
+              (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
+              (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
+              (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+        }
       }
 
       if (match) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index fb898ea..c33d5af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -61,6 +63,24 @@ public interface Server extends Abortable, Stoppable {
   ClusterConnection getClusterConnection();
 
   /**
+   * Returns a reference to the servers' async connection.
+   * <p/>
+   * Important note: this method returns a reference to Connection which is managed by Server
+   * itself, so callers must NOT attempt to close connection obtained.
+   */
+  default AsyncConnection getAsyncConnection() {
+    return getAsyncClusterConnection();
+  }
+
+  /**
+   * Returns a reference to the servers' async cluster connection.
+   * <p/>
+   * Important note: this method returns a reference to Connection which is managed by Server
+   * itself, so callers must NOT attempt to close connection obtained.
+   */
+  AsyncClusterConnection getAsyncClusterConnection();
+
+  /**
    * @return The unique server name for this server.
    */
   ServerName getServerName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/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 0bcef59..52005d6 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
@@ -3008,6 +3008,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.clusterConnection != null) {
       this.clusterConnection.close();
     }
+    if (this.asyncClusterConnection != null) {
+      this.asyncClusterConnection.close();
+    }
   }
 
   public void stopMaster() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/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 13f277b..375b3f8 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
@@ -77,7 +77,9 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -105,7 +107,6 @@ import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
 import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -263,6 +264,11 @@ public class HRegionServer extends HasThread implements
   protected ClusterConnection clusterConnection;
 
   /**
+   * The asynchronous cluster connection to be shared by services.
+   */
+  protected AsyncClusterConnection asyncClusterConnection;
+
+  /**
    * Go here to get table descriptors.
    */
   protected TableDescriptors tableDescriptors;
@@ -776,11 +782,7 @@ public class HRegionServer extends HasThread implements
     return true;
   }
 
-  /**
-   * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
-   * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
-   */
-  private ClusterConnection createClusterConnection() throws IOException {
+  private Configuration unsetClientZookeeperQuorum() {
     Configuration conf = this.conf;
     if (conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) {
       // Use server ZK cluster for server-issued connections, so we clone
@@ -788,11 +790,20 @@ public class HRegionServer extends HasThread implements
       conf = new Configuration(this.conf);
       conf.unset(HConstants.CLIENT_ZOOKEEPER_QUORUM);
     }
+    return conf;
+  }
+
+  /**
+   * Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to the
+   * local server; i.e. a short-circuit Connection. Safe to use going to local or remote server.
+   */
+  private ClusterConnection createClusterConnection() throws IOException {
     // Create a cluster connection that when appropriate, can short-circuit and go directly to the
     // local server if the request is to the local server bypassing RPC. Can be used for both local
     // and remote invocations.
-    ClusterConnection conn = ConnectionUtils.createShortCircuitConnection(conf, null,
-      userProvider.getCurrent(), serverName, rpcServices, rpcServices);
+    ClusterConnection conn =
+      ConnectionUtils.createShortCircuitConnection(unsetClientZookeeperQuorum(), null,
+        userProvider.getCurrent(), serverName, rpcServices, rpcServices);
     // This is used to initialize the batch thread pool inside the connection implementation.
     // When deploy a fresh cluster, we may first use the cluster connection in InitMetaProcedure,
     // which will be executed inside the PEWorker, and then the batch thread pool will inherit the
@@ -826,9 +837,12 @@ public class HRegionServer extends HasThread implements
   /**
    * Setup our cluster connection if not already initialized.
    */
-  protected synchronized void setupClusterConnection() throws IOException {
+  protected final synchronized void setupClusterConnection() throws IOException {
     if (clusterConnection == null) {
       clusterConnection = createClusterConnection();
+      asyncClusterConnection =
+        ClusterConnectionFactory.createAsyncClusterConnection(unsetClientZookeeperQuorum(),
+          new InetSocketAddress(this.rpcServices.isa.getAddress(), 0), userProvider.getCurrent());
     }
   }
 
@@ -842,8 +856,7 @@ public class HRegionServer extends HasThread implements
       initializeZooKeeper();
       setupClusterConnection();
       // Setup RPC client for master communication
-      this.rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
-          this.rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
+      this.rpcClient = asyncClusterConnection.getRpcClient();
     } catch (Throwable t) {
       // Call stop if error or process will stick around for ever since server
       // puts up non-daemon threads.
@@ -1107,7 +1120,15 @@ public class HRegionServer extends HasThread implements
         LOG.warn("Attempt to close server's short circuit ClusterConnection failed.", e);
       }
     }
-
+    if (this.asyncClusterConnection != null) {
+      try {
+        this.asyncClusterConnection.close();
+      } catch (IOException e) {
+        // Although the {@link Closeable} interface throws an {@link
+        // IOException}, in reality, the implementation would never do that.
+        LOG.warn("Attempt to close server's AsyncClusterConnection failed.", e);
+      }
+    }
     // Closing the compactSplit thread before closing meta regions
     if (!this.killed && containsMetaTableRegions()) {
       if (!abortRequested || this.fsOk) {
@@ -3737,9 +3758,9 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public EntityLock regionLock(List<RegionInfo> regionInfos, String description,
-      Abortable abort) throws IOException {
-    return new LockServiceClient(conf, lockStub, clusterConnection.getNonceGenerator())
+  public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort)
+      throws IOException {
+    return new LockServiceClient(conf, lockStub, asyncClusterConnection.getNonceGenerator())
       .regionLock(regionInfos, description, abort);
   }
 
@@ -3843,4 +3864,9 @@ public class HRegionServer extends HasThread implements
       System.exit(1);
     }
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return asyncClusterConnection;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index c7bccb3..7d1245c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -180,5 +181,10 @@ public class ReplicationSyncUp extends Configured implements Tool {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 0e4f241..5205960 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -368,4 +369,8 @@ public class MockRegionServerServices implements RegionServerServices {
   public Optional<MobFileCache> getMobFileCache() {
     return Optional.empty();
   }
+
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index 38dc78d..4fb9716 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -78,7 +78,7 @@ public class TestAsyncNonMetaRegionLocator {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+        registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = new byte[8][];
     for (int i = 111; i < 999; i += 111) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
index c6624e7..1f7d1b6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
@@ -124,7 +124,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+        registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = new AsyncNonMetaRegionLocator(CONN);
     SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
         .toArray(byte[][]::new);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
index 758aa30..0e28f96 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
@@ -96,7 +96,7 @@ public class TestAsyncRegionLocatorTimeout {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+      registry.getClusterId().get(), null, User.getCurrent());
     LOCATOR = CONN.getLocator();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
index a6c2efb..4ce1de8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
@@ -73,7 +73,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent());
+        registry.getClusterId().get(), null, User.getCurrent());
   }
 
   @AfterClass
@@ -164,7 +164,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
           }
         };
     try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(),
-        CONN.registry, CONN.registry.getClusterId().get(), User.getCurrent()) {
+        CONN.registry, CONN.registry.getClusterId().get(), null, User.getCurrent()) {
 
       @Override
       AsyncRegionLocator getLocator() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
index 3008561..e1e55f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java
@@ -85,7 +85,7 @@ public class TestAsyncTableNoncedRetry {
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     ASYNC_CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
-        registry.getClusterId().get(), User.getCurrent()) {
+      registry.getClusterId().get(), null, User.getCurrent()) {
 
       @Override
       public NonceGenerator getNonceGenerator() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/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 9c55f57..3ebad66 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
@@ -473,4 +474,9 @@ public class MockNoopMasterServices implements MasterServices {
   public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() {
     return null;
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index a930d7f..73d53c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -721,4 +722,8 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   public Optional<MobFileCache> getMobFileCache() {
     return Optional.empty();
   }
+
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index 2300f54..77667a7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -349,5 +350,10 @@ public class TestActiveMasterManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 5c8db3e..c5fad32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -279,6 +280,11 @@ public class TestHFileCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index 119194b..fd11ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.HFileLink;
@@ -213,5 +214,10 @@ public class TestHFileLinkCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 247ed01..3286032 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -409,6 +410,11 @@ public class TestLogsCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class FaultyZooKeeperWatcher extends ZKWatcher {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index d162bf3..9791643 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -303,6 +304,11 @@ public class TestReplicationHFileCleaner {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class FaultyZooKeeperWatcher extends ZKWatcher {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index 8c9ce75..4a359e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -862,6 +863,11 @@ public class TestHeapMemoryManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class CustomHeapMemoryTuner implements HeapMemoryTuner {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index cbf932c..5481ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.SplitLogCounters;
 import org.apache.hadoop.hbase.SplitLogTask;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
@@ -159,6 +160,11 @@ public class TestSplitLogWorker {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   private void waitForCounter(LongAdder ctr, long oldval, long newval, long timems)

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/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 0e20252..9e9d1d6 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
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Durability;
@@ -523,6 +524,11 @@ public class TestWALLockup {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   static class DummyWALActionsListener implements WALActionsListener {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index 863d558..62ab265 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -263,5 +264,10 @@ public class TestReplicationTrackerZKImpl {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 86bbb09..427f319 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
@@ -906,5 +907,10 @@ public abstract class TestReplicationSourceManager {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index e4780f1..92c8e54 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -363,6 +364,11 @@ public class TestTokenAuthentication {
     public Connection createConnection(Configuration conf) throws IOException {
       return null;
     }
+
+    @Override
+    public AsyncClusterConnection getAsyncClusterConnection() {
+      return null;
+    }
   }
 
   @Parameters(name = "{index}: rpcServerImpl={0}")

http://git-wip-us.apache.org/repos/asf/hbase/blob/f3caa018/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index c25db01..13212d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
@@ -143,4 +144,9 @@ public class MockServer implements Server {
   public Connection createConnection(Configuration conf) throws IOException {
     return null;
   }
+
+  @Override
+  public AsyncClusterConnection getAsyncClusterConnection() {
+    return null;
+  }
 }


[36/47] hbase git commit: HBASE-21650 Add DDL operation and some other miscellaneous to thrift2

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
index 129ab2e..8450f5b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class THRegionInfo implements org.apache.thrift.TBase<THRegionInfo, THRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionInfo");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
index 94b25ff..b1146e9 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class THRegionLocation implements org.apache.thrift.TBase<THRegionLocation, THRegionLocation._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionLocation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
index 2e50d3d..9569c3f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * to the HBase master or a HBase region server. Also used to return
  * more general HBase error conditions.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TIOError extends TException implements org.apache.thrift.TBase<TIOError, TIOError._Fields>, java.io.Serializable, Cloneable, Comparable<TIOError> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
index 9387429..6734dec 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * A TIllegalArgument exception indicates an illegal or invalid
  * argument was passed into a procedure.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TIllegalArgument extends TException implements org.apache.thrift.TBase<TIllegalArgument, TIllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<TIllegalArgument> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
index 8d62eb5..3663e61 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
 
@@ -50,6 +50,7 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
   private static final org.apache.thrift.protocol.TField ATTRIBUTES_FIELD_DESC = new org.apache.thrift.protocol.TField("attributes", org.apache.thrift.protocol.TType.MAP, (short)4);
   private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
   private static final org.apache.thrift.protocol.TField CELL_VISIBILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("cellVisibility", org.apache.thrift.protocol.TType.STRUCT, (short)6);
+  private static final org.apache.thrift.protocol.TField RETURN_RESULTS_FIELD_DESC = new org.apache.thrift.protocol.TField("returnResults", org.apache.thrift.protocol.TType.BOOL, (short)7);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -66,6 +67,7 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
    */
   public TDurability durability; // optional
   public TCellVisibility cellVisibility; // optional
+  public boolean returnResults; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -77,7 +79,8 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
      * @see TDurability
      */
     DURABILITY((short)5, "durability"),
-    CELL_VISIBILITY((short)6, "cellVisibility");
+    CELL_VISIBILITY((short)6, "cellVisibility"),
+    RETURN_RESULTS((short)7, "returnResults");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -102,6 +105,8 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
           return DURABILITY;
         case 6: // CELL_VISIBILITY
           return CELL_VISIBILITY;
+        case 7: // RETURN_RESULTS
+          return RETURN_RESULTS;
         default:
           return null;
       }
@@ -142,7 +147,9 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.DURABILITY,_Fields.CELL_VISIBILITY};
+  private static final int __RETURNRESULTS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.DURABILITY,_Fields.CELL_VISIBILITY,_Fields.RETURN_RESULTS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -159,6 +166,8 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
     tmpMap.put(_Fields.CELL_VISIBILITY, new org.apache.thrift.meta_data.FieldMetaData("cellVisibility", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCellVisibility.class)));
+    tmpMap.put(_Fields.RETURN_RESULTS, new org.apache.thrift.meta_data.FieldMetaData("returnResults", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TIncrement.class, metaDataMap);
   }
@@ -179,6 +188,7 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
    * Performs a deep copy on <i>other</i>.
    */
   public TIncrement(TIncrement other) {
+    __isset_bitfield = other.__isset_bitfield;
     if (other.isSetRow()) {
       this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row);
     }
@@ -199,6 +209,7 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
     if (other.isSetCellVisibility()) {
       this.cellVisibility = new TCellVisibility(other.cellVisibility);
     }
+    this.returnResults = other.returnResults;
   }
 
   public TIncrement deepCopy() {
@@ -212,6 +223,8 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
     this.attributes = null;
     this.durability = null;
     this.cellVisibility = null;
+    setReturnResultsIsSet(false);
+    this.returnResults = false;
   }
 
   public byte[] getRow() {
@@ -378,6 +391,29 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
     }
   }
 
+  public boolean isReturnResults() {
+    return this.returnResults;
+  }
+
+  public TIncrement setReturnResults(boolean returnResults) {
+    this.returnResults = returnResults;
+    setReturnResultsIsSet(true);
+    return this;
+  }
+
+  public void unsetReturnResults() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RETURNRESULTS_ISSET_ID);
+  }
+
+  /** Returns true if field returnResults is set (has been assigned a value) and false otherwise */
+  public boolean isSetReturnResults() {
+    return EncodingUtils.testBit(__isset_bitfield, __RETURNRESULTS_ISSET_ID);
+  }
+
+  public void setReturnResultsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RETURNRESULTS_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ROW:
@@ -420,6 +456,14 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
       }
       break;
 
+    case RETURN_RESULTS:
+      if (value == null) {
+        unsetReturnResults();
+      } else {
+        setReturnResults((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -440,6 +484,9 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
     case CELL_VISIBILITY:
       return getCellVisibility();
 
+    case RETURN_RESULTS:
+      return isReturnResults();
+
     }
     throw new IllegalStateException();
   }
@@ -461,6 +508,8 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
       return isSetDurability();
     case CELL_VISIBILITY:
       return isSetCellVisibility();
+    case RETURN_RESULTS:
+      return isSetReturnResults();
     }
     throw new IllegalStateException();
   }
@@ -523,6 +572,15 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
         return false;
     }
 
+    boolean this_present_returnResults = true && this.isSetReturnResults();
+    boolean that_present_returnResults = true && that.isSetReturnResults();
+    if (this_present_returnResults || that_present_returnResults) {
+      if (!(this_present_returnResults && that_present_returnResults))
+        return false;
+      if (this.returnResults != that.returnResults)
+        return false;
+    }
+
     return true;
   }
 
@@ -555,6 +613,11 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
     if (present_cellVisibility)
       list.add(cellVisibility);
 
+    boolean present_returnResults = true && (isSetReturnResults());
+    list.add(present_returnResults);
+    if (present_returnResults)
+      list.add(returnResults);
+
     return list.hashCode();
   }
 
@@ -616,6 +679,16 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetReturnResults()).compareTo(other.isSetReturnResults());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetReturnResults()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.returnResults, other.returnResults);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -681,6 +754,12 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
       }
       first = false;
     }
+    if (isSetReturnResults()) {
+      if (!first) sb.append(", ");
+      sb.append("returnResults:");
+      sb.append(this.returnResults);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -709,6 +788,8 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
 
   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
     try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
@@ -797,6 +878,14 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 7: // RETURN_RESULTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.returnResults = iprot.readBool();
+              struct.setReturnResultsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -858,6 +947,11 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
           oprot.writeFieldEnd();
         }
       }
+      if (struct.isSetReturnResults()) {
+        oprot.writeFieldBegin(RETURN_RESULTS_FIELD_DESC);
+        oprot.writeBool(struct.returnResults);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -893,7 +987,10 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
       if (struct.isSetCellVisibility()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetReturnResults()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
       if (struct.isSetAttributes()) {
         {
           oprot.writeI32(struct.attributes.size());
@@ -910,6 +1007,9 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
       if (struct.isSetCellVisibility()) {
         struct.cellVisibility.write(oprot);
       }
+      if (struct.isSetReturnResults()) {
+        oprot.writeBool(struct.returnResults);
+      }
     }
 
     @Override
@@ -929,7 +1029,7 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
         }
       }
       struct.setColumnsIsSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TMap _map84 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
@@ -954,6 +1054,10 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
         struct.cellVisibility.read(iprot);
         struct.setCellVisibilityIsSet(true);
       }
+      if (incoming.get(3)) {
+        struct.returnResults = iprot.readBool();
+        struct.setReturnResultsIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
new file mode 100644
index 0000000..4ce64e2
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TKeepDeletedCells.java
@@ -0,0 +1,63 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum TKeepDeletedCells implements org.apache.thrift.TEnum {
+  /**
+   * Deleted Cells are not retained.
+   */
+  FALSE(0),
+  /**
+   * Deleted Cells are retained until they are removed by other means
+   * such TTL or VERSIONS.
+   * If no TTL is specified or no new versions of delete cells are
+   * written, they are retained forever.
+   */
+  TRUE(1),
+  /**
+   * Deleted Cells are retained until the delete marker expires due to TTL.
+   * This is useful when TTL is combined with MIN_VERSIONS and one
+   * wants to keep a minimum number of versions around but at the same
+   * time remove deleted cells after the TTL.
+   */
+  TTL(2);
+
+  private final int value;
+
+  private TKeepDeletedCells(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TKeepDeletedCells findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return FALSE;
+      case 1:
+        return TRUE;
+      case 2:
+        return TTL;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
new file mode 100644
index 0000000..72d4916
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TNamespaceDescriptor.java
@@ -0,0 +1,554 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+public class TNamespaceDescriptor implements org.apache.thrift.TBase<TNamespaceDescriptor, TNamespaceDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TNamespaceDescriptor> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TNamespaceDescriptor");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TNamespaceDescriptorStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TNamespaceDescriptorTupleSchemeFactory());
+  }
+
+  public String name; // required
+  public Map<String,String> configuration; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    CONFIGURATION((short)2, "configuration");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // CONFIGURATION
+          return CONFIGURATION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CONFIGURATION};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TNamespaceDescriptor.class, metaDataMap);
+  }
+
+  public TNamespaceDescriptor() {
+  }
+
+  public TNamespaceDescriptor(
+    String name)
+  {
+    this();
+    this.name = name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TNamespaceDescriptor(TNamespaceDescriptor other) {
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+    if (other.isSetConfiguration()) {
+      Map<String,String> __this__configuration = new HashMap<String,String>(other.configuration);
+      this.configuration = __this__configuration;
+    }
+  }
+
+  public TNamespaceDescriptor deepCopy() {
+    return new TNamespaceDescriptor(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.configuration = null;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public TNamespaceDescriptor setName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int getConfigurationSize() {
+    return (this.configuration == null) ? 0 : this.configuration.size();
+  }
+
+  public void putToConfiguration(String key, String val) {
+    if (this.configuration == null) {
+      this.configuration = new HashMap<String,String>();
+    }
+    this.configuration.put(key, val);
+  }
+
+  public Map<String,String> getConfiguration() {
+    return this.configuration;
+  }
+
+  public TNamespaceDescriptor setConfiguration(Map<String,String> configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  public void unsetConfiguration() {
+    this.configuration = null;
+  }
+
+  /** Returns true if field configuration is set (has been assigned a value) and false otherwise */
+  public boolean isSetConfiguration() {
+    return this.configuration != null;
+  }
+
+  public void setConfigurationIsSet(boolean value) {
+    if (!value) {
+      this.configuration = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    case CONFIGURATION:
+      if (value == null) {
+        unsetConfiguration();
+      } else {
+        setConfiguration((Map<String,String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    case CONFIGURATION:
+      return getConfiguration();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    case CONFIGURATION:
+      return isSetConfiguration();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TNamespaceDescriptor)
+      return this.equals((TNamespaceDescriptor)that);
+    return false;
+  }
+
+  public boolean equals(TNamespaceDescriptor that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_configuration = true && this.isSetConfiguration();
+    boolean that_present_configuration = true && that.isSetConfiguration();
+    if (this_present_configuration || that_present_configuration) {
+      if (!(this_present_configuration && that_present_configuration))
+        return false;
+      if (!this.configuration.equals(that.configuration))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_configuration = true && (isSetConfiguration());
+    list.add(present_configuration);
+    if (present_configuration)
+      list.add(configuration);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TNamespaceDescriptor other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(other.isSetConfiguration());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetConfiguration()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, other.configuration);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TNamespaceDescriptor(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (isSetConfiguration()) {
+      if (!first) sb.append(", ");
+      sb.append("configuration:");
+      if (this.configuration == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.configuration);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (name == null) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString());
+    }
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TNamespaceDescriptorStandardSchemeFactory implements SchemeFactory {
+    public TNamespaceDescriptorStandardScheme getScheme() {
+      return new TNamespaceDescriptorStandardScheme();
+    }
+  }
+
+  private static class TNamespaceDescriptorStandardScheme extends StandardScheme<TNamespaceDescriptor> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TNamespaceDescriptor struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // CONFIGURATION
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map180 = iprot.readMapBegin();
+                struct.configuration = new HashMap<String,String>(2*_map180.size);
+                String _key181;
+                String _val182;
+                for (int _i183 = 0; _i183 < _map180.size; ++_i183)
+                {
+                  _key181 = iprot.readString();
+                  _val182 = iprot.readString();
+                  struct.configuration.put(_key181, _val182);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setConfigurationIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TNamespaceDescriptor struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.configuration != null) {
+        if (struct.isSetConfiguration()) {
+          oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size()));
+            for (Map.Entry<String, String> _iter184 : struct.configuration.entrySet())
+            {
+              oprot.writeString(_iter184.getKey());
+              oprot.writeString(_iter184.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TNamespaceDescriptorTupleSchemeFactory implements SchemeFactory {
+    public TNamespaceDescriptorTupleScheme getScheme() {
+      return new TNamespaceDescriptorTupleScheme();
+    }
+  }
+
+  private static class TNamespaceDescriptorTupleScheme extends TupleScheme<TNamespaceDescriptor> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TNamespaceDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.name);
+      BitSet optionals = new BitSet();
+      if (struct.isSetConfiguration()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetConfiguration()) {
+        {
+          oprot.writeI32(struct.configuration.size());
+          for (Map.Entry<String, String> _iter185 : struct.configuration.entrySet())
+          {
+            oprot.writeString(_iter185.getKey());
+            oprot.writeString(_iter185.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TNamespaceDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.name = iprot.readString();
+      struct.setNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map186 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.configuration = new HashMap<String,String>(2*_map186.size);
+          String _key187;
+          String _val188;
+          for (int _i189 = 0; _i189 < _map186.size; ++_i189)
+          {
+            _key187 = iprot.readString();
+            _val188 = iprot.readString();
+            struct.configuration.put(_key187, _val188);
+          }
+        }
+        struct.setConfigurationIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
index 552e3d4..ad82c67 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
@@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.io.Serializable, Cloneable, Comparable<TPut> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
index 317f9b5..c19cf89 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
@@ -37,13 +37,14 @@ import org.slf4j.LoggerFactory;
 /**
  * if no Result is found, row and columnValues will not be set.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-07-04")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields>, java.io.Serializable, Cloneable, Comparable<TResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult");
 
   private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField COLUMN_VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("columnValues", org.apache.thrift.protocol.TType.LIST, (short)2);
   private static final org.apache.thrift.protocol.TField STALE_FIELD_DESC = new org.apache.thrift.protocol.TField("stale", org.apache.thrift.protocol.TType.BOOL, (short)3);
+  private static final org.apache.thrift.protocol.TField PARTIAL_FIELD_DESC = new org.apache.thrift.protocol.TField("partial", org.apache.thrift.protocol.TType.BOOL, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -54,12 +55,14 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
   public ByteBuffer row; // optional
   public List<TColumnValue> columnValues; // required
   public boolean stale; // optional
+  public boolean partial; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     ROW((short)1, "row"),
     COLUMN_VALUES((short)2, "columnValues"),
-    STALE((short)3, "stale");
+    STALE((short)3, "stale"),
+    PARTIAL((short)4, "partial");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -80,6 +83,8 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
           return COLUMN_VALUES;
         case 3: // STALE
           return STALE;
+        case 4: // PARTIAL
+          return PARTIAL;
         default:
           return null;
       }
@@ -121,8 +126,9 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
 
   // isset id assignments
   private static final int __STALE_ISSET_ID = 0;
+  private static final int __PARTIAL_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.ROW,_Fields.STALE};
+  private static final _Fields optionals[] = {_Fields.ROW,_Fields.STALE,_Fields.PARTIAL};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -133,6 +139,8 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnValue.class))));
     tmpMap.put(_Fields.STALE, new org.apache.thrift.meta_data.FieldMetaData("stale", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.PARTIAL, new org.apache.thrift.meta_data.FieldMetaData("partial", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TResult.class, metaDataMap);
   }
@@ -140,6 +148,8 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
   public TResult() {
     this.stale = false;
 
+    this.partial = false;
+
   }
 
   public TResult(
@@ -165,6 +175,7 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
       this.columnValues = __this__columnValues;
     }
     this.stale = other.stale;
+    this.partial = other.partial;
   }
 
   public TResult deepCopy() {
@@ -177,6 +188,8 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
     this.columnValues = null;
     this.stale = false;
 
+    this.partial = false;
+
   }
 
   public byte[] getRow() {
@@ -275,6 +288,29 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STALE_ISSET_ID, value);
   }
 
+  public boolean isPartial() {
+    return this.partial;
+  }
+
+  public TResult setPartial(boolean partial) {
+    this.partial = partial;
+    setPartialIsSet(true);
+    return this;
+  }
+
+  public void unsetPartial() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PARTIAL_ISSET_ID);
+  }
+
+  /** Returns true if field partial is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartial() {
+    return EncodingUtils.testBit(__isset_bitfield, __PARTIAL_ISSET_ID);
+  }
+
+  public void setPartialIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PARTIAL_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ROW:
@@ -301,6 +337,14 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
       }
       break;
 
+    case PARTIAL:
+      if (value == null) {
+        unsetPartial();
+      } else {
+        setPartial((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -315,6 +359,9 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
     case STALE:
       return isStale();
 
+    case PARTIAL:
+      return isPartial();
+
     }
     throw new IllegalStateException();
   }
@@ -332,6 +379,8 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
       return isSetColumnValues();
     case STALE:
       return isSetStale();
+    case PARTIAL:
+      return isSetPartial();
     }
     throw new IllegalStateException();
   }
@@ -376,6 +425,15 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
         return false;
     }
 
+    boolean this_present_partial = true && this.isSetPartial();
+    boolean that_present_partial = true && that.isSetPartial();
+    if (this_present_partial || that_present_partial) {
+      if (!(this_present_partial && that_present_partial))
+        return false;
+      if (this.partial != that.partial)
+        return false;
+    }
+
     return true;
   }
 
@@ -398,6 +456,11 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
     if (present_stale)
       list.add(stale);
 
+    boolean present_partial = true && (isSetPartial());
+    list.add(present_partial);
+    if (present_partial)
+      list.add(partial);
+
     return list.hashCode();
   }
 
@@ -439,6 +502,16 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetPartial()).compareTo(other.isSetPartial());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartial()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partial, other.partial);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -482,6 +555,12 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
       sb.append(this.stale);
       first = false;
     }
+    if (isSetPartial()) {
+      if (!first) sb.append(", ");
+      sb.append("partial:");
+      sb.append(this.partial);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -565,6 +644,14 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // PARTIAL
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.partial = iprot.readBool();
+              struct.setPartialIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -604,6 +691,11 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
         oprot.writeBool(struct.stale);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetPartial()) {
+        oprot.writeFieldBegin(PARTIAL_FIELD_DESC);
+        oprot.writeBool(struct.partial);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -635,13 +727,19 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
       if (struct.isSetStale()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetPartial()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetRow()) {
         oprot.writeBinary(struct.row);
       }
       if (struct.isSetStale()) {
         oprot.writeBool(struct.stale);
       }
+      if (struct.isSetPartial()) {
+        oprot.writeBool(struct.partial);
+      }
     }
 
     @Override
@@ -659,7 +757,7 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
         }
       }
       struct.setColumnValuesIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.row = iprot.readBinary();
         struct.setRowIsSet(true);
@@ -668,6 +766,10 @@ public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields
         struct.stale = iprot.readBool();
         struct.setStaleIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.partial = iprot.readBool();
+        struct.setPartialIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
index dfa06ff..a3ab58e 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A TRowMutations object is used to apply a number of Mutations to a single row.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRowMutations._Fields>, java.io.Serializable, Cloneable, Comparable<TRowMutations> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations");
 
@@ -460,14 +460,14 @@ public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRo
           case 2: // MUTATIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list124 = iprot.readListBegin();
-                struct.mutations = new ArrayList<TMutation>(_list124.size);
-                TMutation _elem125;
-                for (int _i126 = 0; _i126 < _list124.size; ++_i126)
+                org.apache.thrift.protocol.TList _list134 = iprot.readListBegin();
+                struct.mutations = new ArrayList<TMutation>(_list134.size);
+                TMutation _elem135;
+                for (int _i136 = 0; _i136 < _list134.size; ++_i136)
                 {
-                  _elem125 = new TMutation();
-                  _elem125.read(iprot);
-                  struct.mutations.add(_elem125);
+                  _elem135 = new TMutation();
+                  _elem135.read(iprot);
+                  struct.mutations.add(_elem135);
                 }
                 iprot.readListEnd();
               }
@@ -500,9 +500,9 @@ public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRo
         oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size()));
-          for (TMutation _iter127 : struct.mutations)
+          for (TMutation _iter137 : struct.mutations)
           {
-            _iter127.write(oprot);
+            _iter137.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -528,9 +528,9 @@ public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRo
       oprot.writeBinary(struct.row);
       {
         oprot.writeI32(struct.mutations.size());
-        for (TMutation _iter128 : struct.mutations)
+        for (TMutation _iter138 : struct.mutations)
         {
-          _iter128.write(oprot);
+          _iter138.write(oprot);
         }
       }
     }
@@ -541,14 +541,14 @@ public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRo
       struct.row = iprot.readBinary();
       struct.setRowIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list129 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.mutations = new ArrayList<TMutation>(_list129.size);
-        TMutation _elem130;
-        for (int _i131 = 0; _i131 < _list129.size; ++_i131)
+        org.apache.thrift.protocol.TList _list139 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.mutations = new ArrayList<TMutation>(_list139.size);
+        TMutation _elem140;
+        for (int _i141 = 0; _i141 < _list139.size; ++_i141)
         {
-          _elem130 = new TMutation();
-          _elem130.read(iprot);
-          struct.mutations.add(_elem130);
+          _elem140 = new TMutation();
+          _elem140.read(iprot);
+          struct.mutations.add(_elem140);
         }
       }
       struct.setMutationsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
index b5ece00..ddcedb4 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * Any timestamps in the columns are ignored but the colFamTimeRangeMap included, use timeRange to select by timestamp.
  * Max versions defaults to 1.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-07-03")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
index 27188e9..65f963d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
 public class TServerName implements org.apache.thrift.TBase<TServerName, TServerName._Fields>, java.io.Serializable, Cloneable, Comparable<TServerName> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7820ba1d/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
new file mode 100644
index 0000000..89a8a5e
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTableDescriptor.java
@@ -0,0 +1,843 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-27")
+public class TTableDescriptor implements org.apache.thrift.TBase<TTableDescriptor, TTableDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TTableDescriptor> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableDescriptor");
+
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)2);
+  private static final org.apache.thrift.protocol.TField ATTRIBUTES_FIELD_DESC = new org.apache.thrift.protocol.TField("attributes", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TTableDescriptorStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TTableDescriptorTupleSchemeFactory());
+  }
+
+  public TTableName tableName; // required
+  public List<TColumnFamilyDescriptor> columns; // optional
+  public Map<ByteBuffer,ByteBuffer> attributes; // optional
+  /**
+   * 
+   * @see TDurability
+   */
+  public TDurability durability; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TABLE_NAME((short)1, "tableName"),
+    COLUMNS((short)2, "columns"),
+    ATTRIBUTES((short)3, "attributes"),
+    /**
+     * 
+     * @see TDurability
+     */
+    DURABILITY((short)4, "durability");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TABLE_NAME
+          return TABLE_NAME;
+        case 2: // COLUMNS
+          return COLUMNS;
+        case 3: // ATTRIBUTES
+          return ATTRIBUTES;
+        case 4: // DURABILITY
+          return DURABILITY;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.COLUMNS,_Fields.ATTRIBUTES,_Fields.DURABILITY};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
+    tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnFamilyDescriptor.class))));
+    tmpMap.put(_Fields.ATTRIBUTES, new org.apache.thrift.meta_data.FieldMetaData("attributes", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
+    tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTableDescriptor.class, metaDataMap);
+  }
+
+  public TTableDescriptor() {
+  }
+
+  public TTableDescriptor(
+    TTableName tableName)
+  {
+    this();
+    this.tableName = tableName;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TTableDescriptor(TTableDescriptor other) {
+    if (other.isSetTableName()) {
+      this.tableName = new TTableName(other.tableName);
+    }
+    if (other.isSetColumns()) {
+      List<TColumnFamilyDescriptor> __this__columns = new ArrayList<TColumnFamilyDescriptor>(other.columns.size());
+      for (TColumnFamilyDescriptor other_element : other.columns) {
+        __this__columns.add(new TColumnFamilyDescriptor(other_element));
+      }
+      this.columns = __this__columns;
+    }
+    if (other.isSetAttributes()) {
+      Map<ByteBuffer,ByteBuffer> __this__attributes = new HashMap<ByteBuffer,ByteBuffer>(other.attributes);
+      this.attributes = __this__attributes;
+    }
+    if (other.isSetDurability()) {
+      this.durability = other.durability;
+    }
+  }
+
+  public TTableDescriptor deepCopy() {
+    return new TTableDescriptor(this);
+  }
+
+  @Override
+  public void clear() {
+    this.tableName = null;
+    this.columns = null;
+    this.attributes = null;
+    this.durability = null;
+  }
+
+  public TTableName getTableName() {
+    return this.tableName;
+  }
+
+  public TTableDescriptor setTableName(TTableName tableName) {
+    this.tableName = tableName;
+    return this;
+  }
+
+  public void unsetTableName() {
+    this.tableName = null;
+  }
+
+  /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableName() {
+    return this.tableName != null;
+  }
+
+  public void setTableNameIsSet(boolean value) {
+    if (!value) {
+      this.tableName = null;
+    }
+  }
+
+  public int getColumnsSize() {
+    return (this.columns == null) ? 0 : this.columns.size();
+  }
+
+  public java.util.Iterator<TColumnFamilyDescriptor> getColumnsIterator() {
+    return (this.columns == null) ? null : this.columns.iterator();
+  }
+
+  public void addToColumns(TColumnFamilyDescriptor elem) {
+    if (this.columns == null) {
+      this.columns = new ArrayList<TColumnFamilyDescriptor>();
+    }
+    this.columns.add(elem);
+  }
+
+  public List<TColumnFamilyDescriptor> getColumns() {
+    return this.columns;
+  }
+
+  public TTableDescriptor setColumns(List<TColumnFamilyDescriptor> columns) {
+    this.columns = columns;
+    return this;
+  }
+
+  public void unsetColumns() {
+    this.columns = null;
+  }
+
+  /** Returns true if field columns is set (has been assigned a value) and false otherwise */
+  public boolean isSetColumns() {
+    return this.columns != null;
+  }
+
+  public void setColumnsIsSet(boolean value) {
+    if (!value) {
+      this.columns = null;
+    }
+  }
+
+  public int getAttributesSize() {
+    return (this.attributes == null) ? 0 : this.attributes.size();
+  }
+
+  public void putToAttributes(ByteBuffer key, ByteBuffer val) {
+    if (this.attributes == null) {
+      this.attributes = new HashMap<ByteBuffer,ByteBuffer>();
+    }
+    this.attributes.put(key, val);
+  }
+
+  public Map<ByteBuffer,ByteBuffer> getAttributes() {
+    return this.attributes;
+  }
+
+  public TTableDescriptor setAttributes(Map<ByteBuffer,ByteBuffer> attributes) {
+    this.attributes = attributes;
+    return this;
+  }
+
+  public void unsetAttributes() {
+    this.attributes = null;
+  }
+
+  /** Returns true if field attributes is set (has been assigned a value) and false otherwise */
+  public boolean isSetAttributes() {
+    return this.attributes != null;
+  }
+
+  public void setAttributesIsSet(boolean value) {
+    if (!value) {
+      this.attributes = null;
+    }
+  }
+
+  /**
+   * 
+   * @see TDurability
+   */
+  public TDurability getDurability() {
+    return this.durability;
+  }
+
+  /**
+   * 
+   * @see TDurability
+   */
+  public TTableDescriptor setDurability(TDurability durability) {
+    this.durability = durability;
+    return this;
+  }
+
+  public void unsetDurability() {
+    this.durability = null;
+  }
+
+  /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+  public boolean isSetDurability() {
+    return this.durability != null;
+  }
+
+  public void setDurabilityIsSet(boolean value) {
+    if (!value) {
+      this.durability = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TABLE_NAME:
+      if (value == null) {
+        unsetTableName();
+      } else {
+        setTableName((TTableName)value);
+      }
+      break;
+
+    case COLUMNS:
+      if (value == null) {
+        unsetColumns();
+      } else {
+        setColumns((List<TColumnFamilyDescriptor>)value);
+      }
+      break;
+
+    case ATTRIBUTES:
+      if (value == null) {
+        unsetAttributes();
+      } else {
+        setAttributes((Map<ByteBuffer,ByteBuffer>)value);
+      }
+      break;
+
+    case DURABILITY:
+      if (value == null) {
+        unsetDurability();
+      } else {
+        setDurability((TDurability)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TABLE_NAME:
+      return getTableName();
+
+    case COLUMNS:
+      return getColumns();
+
+    case ATTRIBUTES:
+      return getAttributes();
+
+    case DURABILITY:
+      return getDurability();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TABLE_NAME:
+      return isSetTableName();
+    case COLUMNS:
+      return isSetColumns();
+    case ATTRIBUTES:
+      return isSetAttributes();
+    case DURABILITY:
+      return isSetDurability();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TTableDescriptor)
+      return this.equals((TTableDescriptor)that);
+    return false;
+  }
+
+  public boolean equals(TTableDescriptor that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_tableName = true && this.isSetTableName();
+    boolean that_present_tableName = true && that.isSetTableName();
+    if (this_present_tableName || that_present_tableName) {
+      if (!(this_present_tableName && that_present_tableName))
+        return false;
+      if (!this.tableName.equals(that.tableName))
+        return false;
+    }
+
+    boolean this_present_columns = true && this.isSetColumns();
+    boolean that_present_columns = true && that.isSetColumns();
+    if (this_present_columns || that_present_columns) {
+      if (!(this_present_columns && that_present_columns))
+        return false;
+      if (!this.columns.equals(that.columns))
+        return false;
+    }
+
+    boolean this_present_attributes = true && this.isSetAttributes();
+    boolean that_present_attributes = true && that.isSetAttributes();
+    if (this_present_attributes || that_present_attributes) {
+      if (!(this_present_attributes && that_present_attributes))
+        return false;
+      if (!this.attributes.equals(that.attributes))
+        return false;
+    }
+
+    boolean this_present_durability = true && this.isSetDurability();
+    boolean that_present_durability = true && that.isSetDurability();
+    if (this_present_durability || that_present_durability) {
+      if (!(this_present_durability && that_present_durability))
+        return false;
+      if (!this.durability.equals(that.durability))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_tableName = true && (isSetTableName());
+    list.add(present_tableName);
+    if (present_tableName)
+      list.add(tableName);
+
+    boolean present_columns = true && (isSetColumns());
+    list.add(present_columns);
+    if (present_columns)
+      list.add(columns);
+
+    boolean present_attributes = true && (isSetAttributes());
+    list.add(present_attributes);
+    if (present_attributes)
+      list.add(attributes);
+
+    boolean present_durability = true && (isSetDurability());
+    list.add(present_durability);
+    if (present_durability)
+      list.add(durability.getValue());
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TTableDescriptor other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetColumns()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAttributes()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDurability()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TTableDescriptor(");
+    boolean first = true;
+
+    sb.append("tableName:");
+    if (this.tableName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tableName);
+    }
+    first = false;
+    if (isSetColumns()) {
+      if (!first) sb.append(", ");
+      sb.append("columns:");
+      if (this.columns == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.columns);
+      }
+      first = false;
+    }
+    if (isSetAttributes()) {
+      if (!first) sb.append(", ");
+      sb.append("attributes:");
+      if (this.attributes == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.attributes);
+      }
+      first = false;
+    }
+    if (isSetDurability()) {
+      if (!first) sb.append(", ");
+      sb.append("durability:");
+      if (this.durability == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.durability);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (tableName == null) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString());
+    }
+    // check for sub-struct validity
+    if (tableName != null) {
+      tableName.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TTableDescriptorStandardSchemeFactory implements SchemeFactory {
+    public TTableDescriptorStandardScheme getScheme() {
+      return new TTableDescriptorStandardScheme();
+    }
+  }
+
+  private static class TTableDescriptorStandardScheme extends StandardScheme<TTableDescriptor> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TTableDescriptor struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TABLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.tableName = new TTableName();
+              struct.tableName.read(iprot);
+              struct.setTableNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COLUMNS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list162 = iprot.readListBegin();
+                struct.columns = new ArrayList<TColumnFamilyDescriptor>(_list162.size);
+                TColumnFamilyDescriptor _elem163;
+                for (int _i164 = 0; _i164 < _list162.size; ++_i164)
+                {
+                  _elem163 = new TColumnFamilyDescriptor();
+                  _elem163.read(iprot);
+                  struct.columns.add(_elem163);
+                }
+                iprot.readListEnd();
+              }
+              struct.setColumnsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ATTRIBUTES
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map165 = iprot.readMapBegin();
+                struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map165.size);
+                ByteBuffer _key166;
+                ByteBuffer _val167;
+                for (int _i168 = 0; _i168 < _map165.size; ++_i168)
+                {
+                  _key166 = iprot.readBinary();
+                  _val167 = iprot.readBinary();
+                  struct.attributes.put(_key166, _val167);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setAttributesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // DURABILITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.durability = org.apache.hadoop.hbase.thrift2.generated.TDurability.findByValue(iprot.readI32());
+              struct.setDurabilityIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TTableDescriptor struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.tableName != null) {
+        oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+        struct.tableName.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.columns != null) {
+        if (struct.isSetColumns()) {
+          oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
+            for (TColumnFamilyDescriptor _iter169 : struct.columns)
+            {
+              _iter169.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.attributes != null) {
+        if (struct.isSetAttributes()) {
+          oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size()));
+            for (Map.Entry<ByteBuffer, ByteBuffer> _iter170 : struct.attributes.entrySet())
+            {
+              oprot.writeBinary(_iter170.getKey());
+              oprot.writeBinary(_iter170.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.durability != null) {
+        if (struct.isSetDurability()) {
+          oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+          oprot.writeI32(struct.durability.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TTableDescriptorTupleSchemeFactory implements SchemeFactory {
+    public TTableDescriptorTupleScheme getScheme() {
+      return new TTableDescriptorTupleScheme();
+    }
+  }
+
+  private static class TTableDescriptorTupleScheme extends TupleScheme<TTableDescriptor> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TTableDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.tableName.write(oprot);
+      BitSet optionals = new BitSet();
+      if (struct.isSetColumns()) {
+        optionals.set(0);
+      }
+      if (struct.isSetAttributes()) {
+        optionals.set(1);
+      }
+      if (struct.isSetDurability()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetColumns()) {
+        {
+          oprot.writeI32(struct.columns.size());
+          for (TColumnFamilyDescriptor _iter171 : struct.columns)
+          {
+            _iter171.write(oprot);
+          }
+        }
+      }
+      if (struct.isSetAttributes()) {
+        {
+          oprot.writeI32(struct.attributes.size());
+          for (Map.Entry<ByteBuffer, ByteBuffer> _iter172 : struct.attributes.entrySet())
+          {
+            oprot.writeBinary(_iter172.getKey());
+            oprot.writeBinary(_iter172.getValue());
+          }
+        }
+      }
+      if (struct.isSetDurability()) {
+        oprot.writeI32(struct.durability.getValue());
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TTableDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.tableName = new TTableName();
+      struct.tableName.read(iprot);
+      struct.setTableNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list173 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.columns = new ArrayList<TColumnFamilyDescriptor>(_list173.size);
+          TColumnFamilyDescriptor _elem174;
+          for (int _i175 = 0; _i175 < _list173.size; ++_i175)
+          {
+            _elem174 = new TColumnFamilyDescriptor();
+            _elem174.read(iprot);
+            struct.columns.add(_elem174);
+          }
+        }
+        struct.setColumnsIsSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map176 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map176.size);
+          ByteBuffer _key177;
+          ByteBuffer _val178;
+          for (int _i179 = 0; _i179 < _map176.size; ++_i179)
+          {
+            _key177 = iprot.readBinary();
+            _val178 = iprot.readBinary();
+            struct.attributes.put(_key177, _val178);
+          }
+        }
+        struct.setAttributesIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.durability = org.apache.hadoop.hbase.thrift2.generated.TDurability.findByValue(iprot.readI32());
+        struct.setDurabilityIsSet(true);
+      }
+    }
+  }
+
+}
+


[47/47] hbase git commit: HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin

Posted by zh...@apache.org.
HBASE-21526 Use AsyncClusterConnection in ServerManager for getRsAdmin


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

Branch: refs/heads/HBASE-21512
Commit: b33b072de945d5272a3d46c06fd278cd64d11142
Parents: a13292d
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 6 21:25:34 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Dec 31 20:34:24 2018 +0800

----------------------------------------------------------------------
 .../hbase/client/AsyncClusterConnection.java    |   6 +
 .../hbase/client/AsyncConnectionImpl.java       |   5 +
 .../hbase/client/AsyncRegionServerAdmin.java    | 210 +++++++++++++++++++
 .../apache/hadoop/hbase/util/FutureUtils.java   |  60 ++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |  13 +-
 .../hadoop/hbase/master/ServerManager.java      |  67 ------
 .../master/procedure/RSProcedureDispatcher.java |  44 ++--
 7 files changed, 320 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b33b072d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
index c7dea25..1327fd7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -27,6 +28,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 public interface AsyncClusterConnection extends AsyncConnection {
 
   /**
+   * Get the admin service for the given region server.
+   */
+  AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName);
+
+  /**
    * Get the nonce generator for this connection.
    */
   NonceGenerator getNonceGenerator();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b33b072d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 79ec54b..b01c03e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -331,4 +331,9 @@ class AsyncConnectionImpl implements AsyncClusterConnection {
     return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
       RETRY_TIMER);
   }
+
+  @Override
+  public AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName) {
+    return new AsyncRegionServerAdmin(serverName, this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b33b072d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
new file mode 100644
index 0000000..9accd89
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionServerAdmin.java
@@ -0,0 +1,210 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+
+/**
+ * A simple wrapper of the {@link AdminService} for a region server, which returns a
+ * {@link CompletableFuture}. This is easier to use, as if you use the raw protobuf interface, you
+ * need to get the result from the {@link RpcCallback}, and if there is an exception, you need to
+ * get it from the {@link RpcController} passed in.
+ * <p/>
+ * Notice that there is no retry, and this is intentional. We have different retry for different
+ * usage for now, if later we want to unify them, we can move the retry logic into this class.
+ */
+@InterfaceAudience.Private
+public class AsyncRegionServerAdmin {
+
+  private final ServerName server;
+
+  private final AsyncConnectionImpl conn;
+
+  AsyncRegionServerAdmin(ServerName server, AsyncConnectionImpl conn) {
+    this.server = server;
+    this.conn = conn;
+  }
+
+  @FunctionalInterface
+  private interface RpcCall<RESP> {
+    void call(AdminService.Interface stub, HBaseRpcController controller, RpcCallback<RESP> done);
+  }
+
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {
+    CompletableFuture<RESP> future = new CompletableFuture<>();
+    HBaseRpcController controller = conn.rpcControllerFactory.newController();
+    try {
+      rpcCall.call(conn.getAdminStub(server), controller, new RpcCallback<RESP>() {
+
+        @Override
+        public void run(RESP resp) {
+          if (controller.failed()) {
+            future.completeExceptionally(controller.getFailed());
+          } else {
+            future.complete(resp);
+          }
+        }
+      });
+    } catch (IOException e) {
+      future.completeExceptionally(e);
+    }
+    return future;
+  }
+
+  public CompletableFuture<GetRegionInfoResponse> getRegionInfo(GetRegionInfoRequest request) {
+    return call((stub, controller, done) -> stub.getRegionInfo(controller, request, done));
+  }
+
+  public CompletableFuture<GetStoreFileResponse> getStoreFile(GetStoreFileRequest request) {
+    return call((stub, controller, done) -> stub.getStoreFile(controller, request, done));
+  }
+
+  public CompletableFuture<GetOnlineRegionResponse> getOnlineRegion(
+      GetOnlineRegionRequest request) {
+    return call((stub, controller, done) -> stub.getOnlineRegion(controller, request, done));
+  }
+
+  public CompletableFuture<OpenRegionResponse> openRegion(OpenRegionRequest request) {
+    return call((stub, controller, done) -> stub.openRegion(controller, request, done));
+  }
+
+  public CompletableFuture<WarmupRegionResponse> warmupRegion(WarmupRegionRequest request) {
+    return call((stub, controller, done) -> stub.warmupRegion(controller, request, done));
+  }
+
+  public CompletableFuture<CloseRegionResponse> closeRegion(CloseRegionRequest request) {
+    return call((stub, controller, done) -> stub.closeRegion(controller, request, done));
+  }
+
+  public CompletableFuture<FlushRegionResponse> flushRegion(FlushRegionRequest request) {
+    return call((stub, controller, done) -> stub.flushRegion(controller, request, done));
+  }
+
+  public CompletableFuture<CompactionSwitchResponse> compactionSwitch(
+      CompactionSwitchRequest request) {
+    return call((stub, controller, done) -> stub.compactionSwitch(controller, request, done));
+  }
+
+  public CompletableFuture<CompactRegionResponse> compactRegion(CompactRegionRequest request) {
+    return call((stub, controller, done) -> stub.compactRegion(controller, request, done));
+  }
+
+  public CompletableFuture<ReplicateWALEntryResponse> replicateWALEntry(
+      ReplicateWALEntryRequest request) {
+    return call((stub, controller, done) -> stub.replicateWALEntry(controller, request, done));
+  }
+
+  public CompletableFuture<ReplicateWALEntryResponse> replay(ReplicateWALEntryRequest request) {
+    return call((stub, controller, done) -> stub.replay(controller, request, done));
+  }
+
+  public CompletableFuture<RollWALWriterResponse> rollWALWriter(RollWALWriterRequest request) {
+    return call((stub, controller, done) -> stub.rollWALWriter(controller, request, done));
+  }
+
+  public CompletableFuture<GetServerInfoResponse> getServerInfo(GetServerInfoRequest request) {
+    return call((stub, controller, done) -> stub.getServerInfo(controller, request, done));
+  }
+
+  public CompletableFuture<StopServerResponse> stopServer(StopServerRequest request) {
+    return call((stub, controller, done) -> stub.stopServer(controller, request, done));
+  }
+
+  public CompletableFuture<UpdateFavoredNodesResponse> updateFavoredNodes(
+      UpdateFavoredNodesRequest request) {
+    return call((stub, controller, done) -> stub.updateFavoredNodes(controller, request, done));
+  }
+
+  public CompletableFuture<UpdateConfigurationResponse> updateConfiguration(
+      UpdateConfigurationRequest request) {
+    return call((stub, controller, done) -> stub.updateConfiguration(controller, request, done));
+  }
+
+  public CompletableFuture<GetRegionLoadResponse> getRegionLoad(GetRegionLoadRequest request) {
+    return call((stub, controller, done) -> stub.getRegionLoad(controller, request, done));
+  }
+
+  public CompletableFuture<ClearCompactionQueuesResponse> clearCompactionQueues(
+      ClearCompactionQueuesRequest request) {
+    return call((stub, controller, done) -> stub.clearCompactionQueues(controller, request, done));
+  }
+
+  public CompletableFuture<ClearRegionBlockCacheResponse> clearRegionBlockCache(
+      ClearRegionBlockCacheRequest request) {
+    return call((stub, controller, done) -> stub.clearRegionBlockCache(controller, request, done));
+  }
+
+  public CompletableFuture<GetSpaceQuotaSnapshotsResponse> getSpaceQuotaSnapshots(
+      GetSpaceQuotaSnapshotsRequest request) {
+    return call((stub, controller, done) -> stub.getSpaceQuotaSnapshots(controller, request, done));
+  }
+
+  public CompletableFuture<ExecuteProceduresResponse> executeProcedures(
+      ExecuteProceduresRequest request) {
+    return call((stub, controller, done) -> stub.executeProcedures(controller, request, done));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b33b072d/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
new file mode 100644
index 0000000..0a72581
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class for processing futures.
+ */
+@InterfaceAudience.Private
+public final class FutureUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FutureUtils.class);
+
+  private FutureUtils() {
+  }
+
+  /**
+   * This is method is used when you do not care the result of an asynchronous operation. Ignoring
+   * the return value of a Future is considered as a bad practice as it may suppress exceptions
+   * thrown from the code that completes the future, so you can use method to log the exceptions
+   * when the future is failed.
+   * <p/>
+   * And the error phone check will always report FutureReturnValueIgnored because every method in
+   * the {@link CompletableFuture} class will return a new {@link CompletableFuture}, so you always
+   * have one future that has not been checked. So we introduce this method and add a suppress
+   * warnings annotation here.
+   */
+  @SuppressWarnings("FutureReturnValueIgnored")
+  public static void ifFail(CompletableFuture<?> future, Consumer<Throwable> action) {
+    future.whenComplete((resp, error) -> {
+      if (error != null) {
+        try {
+          action.accept(error);
+        } catch (Throwable e) {
+          LOG.warn("Failed to process error", error);
+        }
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b33b072d/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 52005d6..b1d478b 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
@@ -193,6 +193,7 @@ import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EncryptionTest;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.HasThread;
@@ -225,6 +226,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
@@ -1937,6 +1939,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     });
   }
 
+  private void warmUpRegion(ServerName server, RegionInfo region) {
+    FutureUtils.ifFail(
+      asyncClusterConnection.getRegionServerAdmin(server)
+        .warmupRegion(RequestConverter.buildWarmupRegionRequest(region)),
+      error -> LOG.warn("Failed to warm up region {} on server {}", region, server, error));
+  }
+
   // Public so can be accessed by tests. Blocks until move is done.
   // Replace with an async implementation from which you can get
   // a success/failure result.
@@ -2008,7 +2017,9 @@ public class HMaster extends HRegionServer implements MasterServices {
       // Warmup the region on the destination before initiating the move. this call
       // is synchronous and takes some time. doing it before the source region gets
       // closed
-      serverManager.sendRegionWarmup(rp.getDestination(), hri);
+      // A region server could reject the close request because it either does not
+      // have the specified region or the region is being split.
+      warmUpRegion(rp.getDestination(), hri);
 
       LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
       Future<byte []> future = this.assignmentManager.moveAsync(rp);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b33b072d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 86d72d1..c26ef6c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -51,12 +50,9 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -159,25 +155,16 @@ public class ServerManager {
   private final ConcurrentNavigableMap<ServerName, ServerMetrics> onlineServers =
     new ConcurrentSkipListMap<>();
 
-  /**
-   * Map of admin interfaces per registered regionserver; these interfaces we use to control
-   * regionservers out on the cluster
-   */
-  private final Map<ServerName, AdminService.BlockingInterface> rsAdmins = new HashMap<>();
-
   /** List of region servers that should not get any more new regions. */
   private final ArrayList<ServerName> drainingServers = new ArrayList<>();
 
   private final MasterServices master;
-  private final ClusterConnection connection;
 
   private final DeadServer deadservers = new DeadServer();
 
   private final long maxSkew;
   private final long warningSkew;
 
-  private final RpcControllerFactory rpcControllerFactory;
-
   /** Listeners that are called on server events. */
   private List<ServerListener> listeners = new CopyOnWriteArrayList<>();
 
@@ -189,8 +176,6 @@ public class ServerManager {
     Configuration c = master.getConfiguration();
     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
-    this.connection = master.getClusterConnection();
-    this.rpcControllerFactory = this.connection == null? null: connection.getRpcControllerFactory();
     persistFlushedSequenceId = c.getBoolean(PERSIST_FLUSHEDSEQUENCEID,
         PERSIST_FLUSHEDSEQUENCEID_DEFAULT);
   }
@@ -438,7 +423,6 @@ public class ServerManager {
   void recordNewServerWithLock(final ServerName serverName, final ServerMetrics sl) {
     LOG.info("Registering regionserver=" + serverName);
     this.onlineServers.put(serverName, sl);
-    this.rsAdmins.remove(serverName);
   }
 
   @VisibleForTesting
@@ -633,7 +617,6 @@ public class ServerManager {
       this.onlineServers.remove(sn);
       onlineServers.notifyAll();
     }
-    this.rsAdmins.remove(sn);
   }
 
   /*
@@ -676,34 +659,6 @@ public class ServerManager {
     return this.drainingServers.add(sn);
   }
 
-  // RPC methods to region servers
-
-  private HBaseRpcController newRpcController() {
-    return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
-  }
-
-  /**
-   * Sends a WARMUP RPC to the specified server to warmup the specified region.
-   * <p>
-   * A region server could reject the close request because it either does not
-   * have the specified region or the region is being split.
-   * @param server server to warmup a region
-   * @param region region to  warmup
-   */
-  public void sendRegionWarmup(ServerName server,
-      RegionInfo region) {
-    if (server == null) return;
-    try {
-      AdminService.BlockingInterface admin = getRsAdmin(server);
-      HBaseRpcController controller = newRpcController();
-      ProtobufUtil.warmupRegion(controller, admin, region);
-    } catch (IOException e) {
-      LOG.error("Received exception in RPC for warmup server:" +
-        server + "region: " + region +
-        "exception: " + e);
-    }
-  }
-
   /**
    * Contacts a region server and waits up to timeout ms
    * to close the region.  This bypasses the active hmaster.
@@ -737,28 +692,6 @@ public class ServerManager {
   }
 
   /**
-   * @param sn
-   * @return Admin interface for the remote regionserver named <code>sn</code>
-   * @throws IOException
-   * @throws RetriesExhaustedException wrapping a ConnectException if failed
-   */
-  public AdminService.BlockingInterface getRsAdmin(final ServerName sn)
-  throws IOException {
-    AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
-    if (admin == null) {
-      LOG.debug("New admin connection to " + sn.toString());
-      if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
-        // A master is also a region server now, see HBASE-10569 for details
-        admin = ((HRegionServer)master).getRSRpcServices();
-      } else {
-        admin = this.connection.getAdmin(sn);
-      }
-      this.rsAdmins.put(sn, admin);
-    }
-    return admin;
-  }
-
-  /**
    * Calculate min necessary to start. This is not an absolute. It is just
    * a friction that will cause us hang around a bit longer waiting on
    * RegionServers to check-in.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b33b072d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 638f9d3..f3ab4b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -18,12 +18,15 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
@@ -37,11 +40,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -159,13 +162,8 @@ public class RSProcedureDispatcher
       this.serverName = serverName;
     }
 
-    protected AdminService.BlockingInterface getRsAdmin() throws IOException {
-      final AdminService.BlockingInterface admin = master.getServerManager().getRsAdmin(serverName);
-      if (admin == null) {
-        throw new IOException("Attempting to send OPEN RPC to server " + getServerName() +
-          " failed because no RPC connection found to this server");
-      }
-      return admin;
+    protected AsyncRegionServerAdmin getRsAdmin() throws IOException {
+      return master.getAsyncClusterConnection().getRegionServerAdmin(serverName);
     }
 
     protected ServerName getServerName() {
@@ -344,9 +342,13 @@ public class RSProcedureDispatcher
     protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
         final ExecuteProceduresRequest request) throws IOException {
       try {
-        return getRsAdmin().executeProcedures(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+        return getRsAdmin().executeProcedures(request).get();
+      } catch (InterruptedException e) {
+        throw (IOException) new InterruptedIOException().initCause(e);
+      } catch (ExecutionException e) {
+        Throwable cause = e.getCause();
+        Throwables.propagateIfPossible(cause, IOException.class);
+        throw new IOException(cause);
       }
     }
 
@@ -407,9 +409,13 @@ public class RSProcedureDispatcher
     private OpenRegionResponse sendRequest(final ServerName serverName,
         final OpenRegionRequest request) throws IOException {
       try {
-        return getRsAdmin().openRegion(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+        return getRsAdmin().openRegion(request).get();
+      } catch (InterruptedException e) {
+        throw (IOException) new InterruptedIOException().initCause(e);
+      } catch (ExecutionException e) {
+        Throwable cause = e.getCause();
+        Throwables.propagateIfPossible(cause, IOException.class);
+        throw new IOException(cause);
       }
     }
 
@@ -453,9 +459,13 @@ public class RSProcedureDispatcher
     private CloseRegionResponse sendRequest(final ServerName serverName,
         final CloseRegionRequest request) throws IOException {
       try {
-        return getRsAdmin().closeRegion(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+        return getRsAdmin().closeRegion(request).get();
+      } catch (InterruptedException e) {
+        throw (IOException) new InterruptedIOException().initCause(e);
+      } catch (ExecutionException e) {
+        Throwable cause = e.getCause();
+        Throwables.propagateIfPossible(cause, IOException.class);
+        throw new IOException(cause);
       }
     }
 


[02/47] hbase git commit: HBASE-21453 Convert ReadOnlyZKClient to DEBUG instead of INFO

Posted by zh...@apache.org.
HBASE-21453 Convert ReadOnlyZKClient to DEBUG instead of INFO


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

Branch: refs/heads/HBASE-21512
Commit: f88224ee34ba2c23f794ec1219ffd93783b20e51
Parents: b09b87d
Author: Sakthi <ja...@cloudera.com>
Authored: Thu Nov 29 18:52:50 2018 -0800
Committer: Peter Somogyi <ps...@apache.org>
Committed: Tue Dec 11 08:18:02 2018 +0100

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/f88224ee/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
index fc2d5f0..09f8984 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
@@ -136,7 +136,7 @@ public final class ReadOnlyZKClient implements Closeable {
     this.retryIntervalMs =
         conf.getInt(RECOVERY_RETRY_INTERVAL_MILLIS, DEFAULT_RECOVERY_RETRY_INTERVAL_MILLIS);
     this.keepAliveTimeMs = conf.getInt(KEEPALIVE_MILLIS, DEFAULT_KEEPALIVE_MILLIS);
-    LOG.info(
+    LOG.debug(
       "Connect {} to {} with session timeout={}ms, retries {}, " +
         "retry interval {}ms, keepAlive={}ms",
       getId(), connectString, sessionTimeoutMs, maxRetries, retryIntervalMs, keepAliveTimeMs);
@@ -347,7 +347,7 @@ public final class ReadOnlyZKClient implements Closeable {
   @Override
   public void close() {
     if (closed.compareAndSet(false, true)) {
-      LOG.info("Close zookeeper connection {} to {}", getId(), connectString);
+      LOG.debug("Close zookeeper connection {} to {}", getId(), connectString);
       tasks.add(CLOSE);
     }
   }


[03/47] hbase git commit: HBASE-21568 Use CacheConfig.DISABLED where we don't expect to have blockcache running

Posted by zh...@apache.org.
HBASE-21568 Use CacheConfig.DISABLED where we don't expect to have blockcache running

This includes removing the "old way" of disabling blockcache in favor of the
new API.

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

Branch: refs/heads/HBASE-21512
Commit: 67d6d5084cf8fc094cda4bd3f091d8a0a9cb1d3e
Parents: f88224e
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 7 17:18:49 2018 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Tue Dec 11 10:02:18 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java  | 6 ++----
 .../src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java  | 4 +---
 .../org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java   | 2 +-
 .../org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java    | 6 +++---
 .../java/org/apache/hadoop/hbase/util/CompressionTest.java     | 2 +-
 .../src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java  | 5 ++---
 .../apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java  | 2 +-
 7 files changed, 11 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/67d6d508/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index c911e8c..274a506 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -414,8 +414,6 @@ public class HFileOutputFormat2
         DataBlockEncoding encoding = overriddenEncoding;
         encoding = encoding == null ? datablockEncodingMap.get(tableAndFamily) : encoding;
         encoding = encoding == null ? DataBlockEncoding.NONE : encoding;
-        Configuration tempConf = new Configuration(conf);
-        tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
         HFileContextBuilder contextBuilder = new HFileContextBuilder()
                                     .withCompression(compression)
                                     .withChecksumType(HStore.getChecksumType(conf))
@@ -430,12 +428,12 @@ public class HFileOutputFormat2
         HFileContext hFileContext = contextBuilder.build();
         if (null == favoredNodes) {
           wl.writer =
-              new StoreFileWriter.Builder(conf, new CacheConfig(tempConf), fs)
+              new StoreFileWriter.Builder(conf, CacheConfig.DISABLED, fs)
                   .withOutputDir(familydir).withBloomType(bloomType)
                   .withComparator(CellComparator.getInstance()).withFileContext(hFileContext).build();
         } else {
           wl.writer =
-              new StoreFileWriter.Builder(conf, new CacheConfig(tempConf), new HFileSystem(fs))
+              new StoreFileWriter.Builder(conf, CacheConfig.DISABLED, new HFileSystem(fs))
                   .withOutputDir(familydir).withBloomType(bloomType)
                   .withComparator(CellComparator.getInstance()).withFileContext(hFileContext)
                   .withFavoredNodes(favoredNodes).build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/67d6d508/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 5bcaa17..78ebedc 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
@@ -356,9 +356,7 @@ public class HFile {
    */
   public static final WriterFactory getWriterFactoryNoCache(Configuration
        conf) {
-    Configuration tempConf = new Configuration(conf);
-    tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
-    return HFile.getWriterFactory(conf, new CacheConfig(tempConf));
+    return HFile.getWriterFactory(conf, CacheConfig.DISABLED);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/67d6d508/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 82e881b..5a6f6c1 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
@@ -309,7 +309,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       return -2;
     }
 
-    HFile.Reader reader = HFile.createReader(fs, file, new CacheConfig(getConf()), true, getConf());
+    HFile.Reader reader = HFile.createReader(fs, file, CacheConfig.DISABLED, true, getConf());
 
     Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/67d6d508/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
index e027ac6..3320b1f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
@@ -710,7 +710,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     Path hfilePath = item.getFilePath();
     Optional<byte[]> first, last;
     try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath,
-      new CacheConfig(getConf()), true, getConf())) {
+      CacheConfig.DISABLED, true, getConf())) {
       hfr.loadFileInfo();
       first = hfr.getFirstRowKey();
       last = hfr.getLastRowKey();
@@ -847,7 +847,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
           throws IOException {
         Path hfile = hfileStatus.getPath();
         try (HFile.Reader reader =
-            HFile.createReader(fs, hfile, new CacheConfig(getConf()), true, getConf())) {
+            HFile.createReader(fs, hfile, CacheConfig.DISABLED, true, getConf())) {
           if (builder.getCompressionType() != reader.getFileContext().getCompression()) {
             builder.setCompressionType(reader.getFileContext().getCompression());
             LOG.info("Setting compression " + reader.getFileContext().getCompression().name() +
@@ -1083,7 +1083,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   private static void copyHFileHalf(Configuration conf, Path inFile, Path outFile,
       Reference reference, ColumnFamilyDescriptor familyDescriptor) throws IOException {
     FileSystem fs = inFile.getFileSystem(conf);
-    CacheConfig cacheConf = new CacheConfig(conf);
+    CacheConfig cacheConf = CacheConfig.DISABLED;
     HalfStoreFileReader halfReader = null;
     StoreFileWriter halfWriter = null;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/67d6d508/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 b6af8a5..dcdd12e 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
@@ -134,7 +134,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), true, conf);
+    HFile.Reader reader = HFile.createReader(fs, path, CacheConfig.DISABLED, true, conf);
     try {
       reader.loadFileInfo();
       HFileScanner scanner = reader.getScanner(false, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/67d6d508/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 14706c5..8176942 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
@@ -922,7 +922,7 @@ public class HBaseFsck extends Configured implements Closeable {
             // For all the stores in this column family.
             for (FileStatus storeFile : storeFiles) {
               HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(),
-                new CacheConfig(getConf()), true, getConf());
+                CacheConfig.DISABLED, true, getConf());
               if ((reader.getFirstKey() != null)
                   && ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
                       ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey().get()).getKey()) > 0))) {
@@ -1025,8 +1025,7 @@ public class HBaseFsck extends Configured implements Closeable {
         byte[] start, end;
         HFile.Reader hf = null;
         try {
-          CacheConfig cacheConf = new CacheConfig(getConf());
-          hf = HFile.createReader(fs, hfile.getPath(), cacheConf, true, getConf());
+          hf = HFile.createReader(fs, hfile.getPath(), CacheConfig.DISABLED, true, getConf());
           hf.loadFileInfo();
           Optional<Cell> startKv = hf.getFirstKey();
           start = CellUtil.cloneRow(startKv.get());

http://git-wip-us.apache.org/repos/asf/hbase/blob/67d6d508/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 e937fa5..41f3cde 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
@@ -82,7 +82,7 @@ public class HFileCorruptionChecker {
       boolean quarantine) throws IOException {
     this.conf = conf;
     this.fs = FileSystem.get(conf);
-    this.cacheConf = new CacheConfig(conf);
+    this.cacheConf = CacheConfig.DISABLED;
     this.executor = executor;
     this.inQuarantineMode = quarantine;
   }


[15/47] hbase git commit: HBASE-21514 Refactor CacheConfig

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 6242d36..13f277b 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
@@ -36,6 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.Timer;
@@ -98,7 +99,7 @@ import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
@@ -114,7 +115,7 @@ import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.mob.MobCacheConfig;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
@@ -410,10 +411,10 @@ public class HRegionServer extends HasThread implements
 
   private final RegionServerAccounting regionServerAccounting;
 
-  // Cache configuration and block cache reference
-  protected CacheConfig cacheConfig;
-  // Cache configuration for mob
-  final MobCacheConfig mobCacheConfig;
+  // Block cache
+  private BlockCache blockCache;
+  // The cache for mob files
+  private MobFileCache mobFileCache;
 
   /** The health check chore. */
   private HealthCheckChore healthCheckChore;
@@ -591,12 +592,12 @@ public class HRegionServer extends HasThread implements
 
       boolean isMasterNotCarryTable =
           this instanceof HMaster && !LoadBalancer.isTablesOnMaster(conf);
-      // no need to instantiate global block cache when master not carry table
+
+      // no need to instantiate block cache and mob file cache when master not carry table
       if (!isMasterNotCarryTable) {
-        CacheConfig.instantiateBlockCache(conf);
+        blockCache = BlockCacheFactory.createBlockCache(conf);
+        mobFileCache = new MobFileCache(conf);
       }
-      cacheConfig = new CacheConfig(conf);
-      mobCacheConfig = new MobCacheConfig(conf);
 
       uncaughtExceptionHandler = new UncaughtExceptionHandler() {
         @Override
@@ -1062,10 +1063,12 @@ public class HRegionServer extends HasThread implements
       }
     }
     // Send cache a shutdown.
-    if (cacheConfig != null && cacheConfig.isBlockCacheEnabled()) {
-      cacheConfig.getBlockCache().shutdown();
+    if (blockCache != null) {
+      blockCache.shutdown();
+    }
+    if (mobFileCache != null) {
+      mobFileCache.shutdown();
     }
-    mobCacheConfig.getMobFileCache().shutdown();
 
     if (movedRegionsCleaner != null) {
       movedRegionsCleaner.stop("Region Server stopping");
@@ -1607,9 +1610,9 @@ public class HRegionServer extends HasThread implements
   }
 
   private void startHeapMemoryManager() {
-    this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher, this,
-        this.regionServerAccounting);
-    if (this.hMemManager != null) {
+    if (this.blockCache != null) {
+      this.hMemManager =
+          new HeapMemoryManager(this.blockCache, this.cacheFlusher, this, regionServerAccounting);
       this.hMemManager.start(getChoreService());
     }
   }
@@ -3614,10 +3617,23 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * @return The cache config instance used by the regionserver.
+   * May be null if this is a master which not carry table.
+   *
+   * @return The block cache instance used by the regionserver.
+   */
+  @Override
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.ofNullable(this.blockCache);
+  }
+
+  /**
+   * May be null if this is a master which not carry table.
+   *
+   * @return The cache for mob files used by the regionserver.
    */
-  public CacheConfig getCacheConfig() {
-    return this.cacheConfig;
+  @Override
+  public Optional<MobFileCache> getMobFileCache() {
+    return Optional.ofNullable(this.mobFileCache);
   }
 
   /**
@@ -3646,7 +3662,6 @@ public class HRegionServer extends HasThread implements
   }
 
   public CacheEvictionStats clearRegionBlockCache(Region region) {
-    BlockCache blockCache = this.getCacheConfig().getBlockCache();
     long evictedBlocks = 0;
 
     for(Store store : region.getStores()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 032dc5f..b3e5b97 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
@@ -377,7 +377,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
    * @param family The current column family.
    */
   protected void createCacheConf(final ColumnFamilyDescriptor family) {
-    this.cacheConf = new CacheConfig(conf, family);
+    this.cacheConf = new CacheConfig(conf, family, region.getBlockCache());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
index c32fce2..a96417d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
@@ -30,13 +30,14 @@ import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Server;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
 import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
@@ -105,20 +106,11 @@ public class HeapMemoryManager {
 
   private List<HeapMemoryTuneObserver> tuneObservers = new ArrayList<>();
 
-  public static HeapMemoryManager create(Configuration conf, FlushRequester memStoreFlusher,
-      Server server, RegionServerAccounting regionServerAccounting) {
-    ResizableBlockCache lruCache = CacheConfig.getOnHeapCache(conf);
-    if (lruCache != null) {
-      return new HeapMemoryManager(lruCache, memStoreFlusher, server, regionServerAccounting);
-    }
-    return null;
-  }
-
   @VisibleForTesting
-  HeapMemoryManager(ResizableBlockCache blockCache, FlushRequester memStoreFlusher,
+  HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher,
                 Server server, RegionServerAccounting regionServerAccounting) {
     Configuration conf = server.getConfiguration();
-    this.blockCache = blockCache;
+    this.blockCache = toResizableBlockCache(blockCache);
     this.memStoreFlusher = memStoreFlusher;
     this.server = server;
     this.regionServerAccounting = regionServerAccounting;
@@ -130,6 +122,14 @@ public class HeapMemoryManager {
     metricsHeapMemoryManager = new MetricsHeapMemoryManager();
   }
 
+  private ResizableBlockCache toResizableBlockCache(BlockCache blockCache) {
+    if (blockCache instanceof CombinedBlockCache) {
+      return (ResizableBlockCache) ((CombinedBlockCache) blockCache).getOnHeapCache();
+    } else {
+      return (ResizableBlockCache) blockCache;
+    }
+  }
+
   private boolean doInit(Configuration conf) {
     boolean tuningEnabled = true;
     globalMemStorePercent = MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index b38c3e0..33a6ee0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
+import java.util.Optional;
 import java.util.OptionalDouble;
 import java.util.OptionalLong;
 import java.util.concurrent.ScheduledExecutorService;
@@ -32,9 +33,8 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
-import org.apache.hadoop.hbase.mob.MobCacheConfig;
+import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
 import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -59,8 +59,11 @@ class MetricsRegionServerWrapperImpl
   private final HRegionServer regionServer;
   private final MetricsWALSource metricsWALSource;
 
-  private BlockCache blockCache;
-  private MobFileCache mobFileCache;
+  private Optional<BlockCache> blockCache;
+  private Optional<MobFileCache> mobFileCache;
+  private Optional<CacheStats> cacheStats;
+  private Optional<CacheStats> l1Stats = Optional.empty();
+  private Optional<CacheStats> l2Stats = Optional.empty();
 
   private volatile long numStores = 0;
   private volatile long numWALFiles = 0;
@@ -112,9 +115,6 @@ class MetricsRegionServerWrapperImpl
   private volatile long blockedRequestsCount = 0L;
   private volatile long averageRegionSize = 0L;
 
-  private CacheStats cacheStats;
-  private CacheStats l1Stats = null;
-  private CacheStats l2Stats = null;
   private ScheduledExecutorService executor;
   private Runnable runnable;
   private long period;
@@ -149,34 +149,26 @@ class MetricsRegionServerWrapperImpl
     }
   }
 
-  /**
-   * It's possible that due to threading the block cache could not be initialized
-   * yet (testing multiple region servers in one jvm).  So we need to try and initialize
-   * the blockCache and cacheStats reference multiple times until we succeed.
-   */
-  private synchronized  void initBlockCache() {
-    CacheConfig cacheConfig = this.regionServer.cacheConfig;
-    if (cacheConfig != null) {
-      l1Stats = cacheConfig.getOnHeapCacheStats();
-      l2Stats = cacheConfig.getL2CacheStats();
-      if (this.blockCache == null) {
-        this.blockCache = cacheConfig.getBlockCache();
+  private void initBlockCache() {
+    this.blockCache = this.regionServer.getBlockCache();
+    this.cacheStats = this.blockCache.map(BlockCache::getStats);
+    if (this.cacheStats.isPresent()) {
+      if (this.cacheStats.get() instanceof CombinedBlockCache.CombinedCacheStats) {
+        l1Stats = Optional
+            .of(((CombinedBlockCache.CombinedCacheStats) this.cacheStats.get()).getLruCacheStats());
+        l2Stats = Optional.of(((CombinedBlockCache.CombinedCacheStats) this.cacheStats.get())
+            .getBucketCacheStats());
+      } else {
+        l1Stats = this.cacheStats;
       }
     }
-
-    if (this.blockCache != null && this.cacheStats == null) {
-      this.cacheStats = blockCache.getStats();
-    }
   }
 
   /**
    * Initializes the mob file cache.
    */
-  private synchronized void initMobFileCache() {
-    MobCacheConfig mobCacheConfig = this.regionServer.mobCacheConfig;
-    if (mobCacheConfig != null && this.mobFileCache == null) {
-      this.mobFileCache = mobCacheConfig.getMobFileCache();
-    }
+  private void initMobFileCache() {
+    this.mobFileCache = this.regionServer.getMobFileCache();
   }
 
   @Override
@@ -281,10 +273,7 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getBlockCacheCount() {
-    if (this.blockCache == null) {
-      return 0;
-    }
-    return this.blockCache.getBlockCount();
+    return this.blockCache.map(BlockCache::getBlockCount).orElse(0L);
   }
 
   @Override
@@ -294,74 +283,47 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getBlockCacheSize() {
-    if (this.blockCache == null) {
-      return 0;
-    }
-    return this.blockCache.getCurrentSize();
+    return this.blockCache.map(BlockCache::getCurrentSize).orElse(0L);
   }
 
   @Override
   public long getBlockCacheFreeSize() {
-    if (this.blockCache == null) {
-      return 0;
-    }
-    return this.blockCache.getFreeSize();
+    return this.blockCache.map(BlockCache::getFreeSize).orElse(0L);
   }
 
   @Override
   public long getBlockCacheHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getHitCount();
+    return this.cacheStats.map(CacheStats::getHitCount).orElse(0L);
   }
 
   @Override
   public long getBlockCachePrimaryHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getPrimaryHitCount();
+    return this.cacheStats.map(CacheStats::getPrimaryHitCount).orElse(0L);
   }
 
   @Override
   public long getBlockCacheMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getMissCount();
+    return this.cacheStats.map(CacheStats::getMissCount).orElse(0L);
   }
 
   @Override
   public long getBlockCachePrimaryMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getPrimaryMissCount();
+    return this.cacheStats.map(CacheStats::getPrimaryMissCount).orElse(0L);
   }
 
   @Override
   public long getBlockCacheEvictedCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getEvictedCount();
+    return this.cacheStats.map(CacheStats::getEvictedCount).orElse(0L);
   }
 
   @Override
   public long getBlockCachePrimaryEvictedCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getPrimaryEvictedCount();
+    return this.cacheStats.map(CacheStats::getPrimaryEvictedCount).orElse(0L);
   }
 
   @Override
   public double getBlockCacheHitPercent() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    double ratio = this.cacheStats.getHitRatio();
+    double ratio = this.cacheStats.map(CacheStats::getHitRatio).orElse(0.0);
     if (Double.isNaN(ratio)) {
       ratio = 0;
     }
@@ -370,12 +332,7 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public double getBlockCacheHitCachingPercent() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-
-    double ratio = this.cacheStats.getHitCachingRatio();
-
+    double ratio = this.cacheStats.map(CacheStats::getHitCachingRatio).orElse(0.0);
     if (Double.isNaN(ratio)) {
       ratio = 0;
     }
@@ -384,74 +341,47 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getBlockCacheFailedInsertions() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getFailedInserts();
+    return this.cacheStats.map(CacheStats::getFailedInserts).orElse(0L);
   }
 
   @Override
   public long getL1CacheHitCount() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getHitCount();
+    return this.l1Stats.map(CacheStats::getHitCount).orElse(0L);
   }
 
   @Override
   public long getL1CacheMissCount() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getMissCount();
+    return this.l1Stats.map(CacheStats::getMissCount).orElse(0L);
   }
 
   @Override
   public double getL1CacheHitRatio() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getHitRatio();
+    return this.l1Stats.map(CacheStats::getHitRatio).orElse(0.0);
   }
 
   @Override
   public double getL1CacheMissRatio() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getMissRatio();
+    return this.l1Stats.map(CacheStats::getMissRatio).orElse(0.0);
   }
 
   @Override
   public long getL2CacheHitCount() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getHitCount();
+    return this.l2Stats.map(CacheStats::getHitCount).orElse(0L);
   }
 
   @Override
   public long getL2CacheMissCount() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getMissCount();
+    return this.l2Stats.map(CacheStats::getMissCount).orElse(0L);
   }
 
   @Override
   public double getL2CacheHitRatio() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getHitRatio();
+    return this.l2Stats.map(CacheStats::getHitRatio).orElse(0.0);
   }
 
   @Override
   public double getL2CacheMissRatio() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getMissRatio();
+    return this.l2Stats.map(CacheStats::getMissRatio).orElse(0.0);
   }
 
   @Override public void forceRecompute() {
@@ -741,9 +671,6 @@ class MetricsRegionServerWrapperImpl
     @Override
     synchronized public void run() {
       try {
-        initBlockCache();
-        initMobFileCache();
-
         HDFSBlocksDistribution hdfsBlocksDistribution =
             new HDFSBlocksDistribution();
         HDFSBlocksDistribution hdfsBlocksDistributionSecondaryRegions =
@@ -945,12 +872,14 @@ class MetricsRegionServerWrapperImpl
         mobFlushedCellsSize = tempMobFlushedCellsSize;
         mobScanCellsCount = tempMobScanCellsCount;
         mobScanCellsSize = tempMobScanCellsSize;
-        mobFileCacheAccessCount = mobFileCache.getAccessCount();
-        mobFileCacheMissCount = mobFileCache.getMissCount();
-        mobFileCacheHitRatio = Double.
-            isNaN(mobFileCache.getHitRatio())?0:mobFileCache.getHitRatio();
-        mobFileCacheEvictedCount = mobFileCache.getEvictedFileCount();
-        mobFileCacheCount = mobFileCache.getCacheSize();
+        mobFileCacheAccessCount = mobFileCache.map(MobFileCache::getAccessCount).orElse(0L);
+        mobFileCacheMissCount = mobFileCache.map(MobFileCache::getMissCount).orElse(0L);
+        mobFileCacheHitRatio = mobFileCache.map(MobFileCache::getHitRatio).orElse(0.0);
+        if (Double.isNaN(mobFileCacheHitRatio)) {
+          mobFileCacheHitRatio = 0.0;
+        }
+        mobFileCacheEvictedCount = mobFileCache.map(MobFileCache::getEvictedFileCount).orElse(0L);
+        mobFileCacheCount = mobFileCache.map(MobFileCache::getCacheSize).orElse(0);
         blockedRequestsCount = tempBlockedRequestsCount;
       } catch (Throwable e) {
         LOG.warn("Caught exception! Will suppress and retry.", e);
@@ -980,161 +909,101 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getDataMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDataMissCount();
+    return this.cacheStats.map(CacheStats::getDataMissCount).orElse(0L);
   }
 
   @Override
   public long getLeafIndexMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getLeafIndexMissCount();
+    return this.cacheStats.map(CacheStats::getLeafIndexMissCount).orElse(0L);
   }
 
   @Override
   public long getBloomChunkMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getBloomChunkMissCount();
+    return this.cacheStats.map(CacheStats::getBloomChunkMissCount).orElse(0L);
   }
 
   @Override
   public long getMetaMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getMetaMissCount();
+    return this.cacheStats.map(CacheStats::getMetaMissCount).orElse(0L);
   }
 
   @Override
   public long getRootIndexMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getRootIndexMissCount();
+    return this.cacheStats.map(CacheStats::getRootIndexMissCount).orElse(0L);
   }
 
   @Override
   public long getIntermediateIndexMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getIntermediateIndexMissCount();
+    return this.cacheStats.map(CacheStats::getIntermediateIndexMissCount).orElse(0L);
   }
 
   @Override
   public long getFileInfoMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getFileInfoMissCount();
+    return this.cacheStats.map(CacheStats::getFileInfoMissCount).orElse(0L);
   }
 
   @Override
   public long getGeneralBloomMetaMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getGeneralBloomMetaMissCount();
+    return this.cacheStats.map(CacheStats::getGeneralBloomMetaMissCount).orElse(0L);
   }
 
   @Override
   public long getDeleteFamilyBloomMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDeleteFamilyBloomMissCount();
+    return this.cacheStats.map(CacheStats::getDeleteFamilyBloomMissCount).orElse(0L);
   }
 
   @Override
   public long getTrailerMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getTrailerMissCount();
+    return this.cacheStats.map(CacheStats::getTrailerMissCount).orElse(0L);
   }
 
   @Override
   public long getDataHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDataHitCount();
+    return this.cacheStats.map(CacheStats::getDataHitCount).orElse(0L);
   }
 
   @Override
   public long getLeafIndexHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getLeafIndexHitCount();
+    return this.cacheStats.map(CacheStats::getLeafIndexHitCount).orElse(0L);
   }
 
   @Override
   public long getBloomChunkHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getBloomChunkHitCount();
+    return this.cacheStats.map(CacheStats::getBloomChunkHitCount).orElse(0L);
   }
 
   @Override
   public long getMetaHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getMetaHitCount();
+    return this.cacheStats.map(CacheStats::getMetaHitCount).orElse(0L);
   }
 
   @Override
   public long getRootIndexHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getRootIndexHitCount();
+    return this.cacheStats.map(CacheStats::getRootIndexHitCount).orElse(0L);
   }
 
   @Override
   public long getIntermediateIndexHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getIntermediateIndexHitCount();
+    return this.cacheStats.map(CacheStats::getIntermediateIndexHitCount).orElse(0L);
   }
 
   @Override
   public long getFileInfoHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getFileInfoHitCount();
+    return this.cacheStats.map(CacheStats::getFileInfoHitCount).orElse(0L);
   }
 
   @Override
   public long getGeneralBloomMetaHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getGeneralBloomMetaHitCount();
+    return this.cacheStats.map(CacheStats::getGeneralBloomMetaHitCount).orElse(0L);
   }
 
   @Override
   public long getDeleteFamilyBloomHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDeleteFamilyBloomHitCount();
+    return this.cacheStats.map(CacheStats::getDeleteFamilyBloomHitCount).orElse(0L);
   }
 
   @Override
   public long getTrailerHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getTrailerHitCount();
+    return this.cacheStats.map(CacheStats::getTrailerHitCount).orElse(0L);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index df84dcf..31df37a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.exceptions.ScannerResetException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
@@ -3683,7 +3684,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         stats.addException(region.getRegionInfo().getRegionName(), e);
       }
     }
-    stats.withMaxCacheSize(regionServer.getCacheConfig().getBlockCache().getMaxSize());
+    stats.withMaxCacheSize(regionServer.getBlockCache().map(BlockCache::getMaxSize).orElse(0L));
     return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 37a3606..e0638ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Server;
@@ -31,7 +32,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionSizeStore;
@@ -266,4 +269,14 @@ public interface RegionServerServices extends Server, MutableOnlineRegions, Favo
    * @return Return table descriptors implementation.
    */
   TableDescriptors getTableDescriptors();
-}
+
+  /**
+   * @return The block cache instance.
+   */
+  Optional<BlockCache> getBlockCache();
+
+  /**
+   * @return The cache for mob files.
+   */
+  Optional<MobFileCache> getMobFileCache();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 31a7cad..0cd5a22 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
@@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.ChecksumUtil;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -104,6 +105,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
 import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -2512,6 +2514,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     return new WALFactory(confForWAL, "hregion-" + RandomStringUtils.randomNumeric(8)).getWAL(hri);
   }
 
+
   /**
    * Create a region with it's own WAL. Be sure to call
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
@@ -2526,6 +2529,31 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
    */
   public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
+      final Configuration conf, final TableDescriptor htd, BlockCache blockCache)
+      throws IOException {
+    HRegion region = createRegionAndWAL(info, rootDir, conf, htd, false);
+    region.setBlockCache(blockCache);
+    region.initialize();
+    return region;
+  }
+  /**
+   * Create a region with it's own WAL. Be sure to call
+   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
+   */
+  public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
+      final Configuration conf, final TableDescriptor htd, MobFileCache mobFileCache)
+      throws IOException {
+    HRegion region = createRegionAndWAL(info, rootDir, conf, htd, false);
+    region.setMobFileCache(mobFileCache);
+    region.initialize();
+    return region;
+  }
+
+  /**
+   * Create a region with it's own WAL. Be sure to call
+   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
+   */
+  public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
       final Configuration conf, final TableDescriptor htd, boolean initialize)
       throws IOException {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
@@ -4037,17 +4065,21 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       + " on server " + server);
   }
 
-  public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd)
-      throws IOException {
-    TableDescriptor td
-        = TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
-            .setColumnFamily(cd)
-            .build();
-    HRegionInfo info =
-        new HRegionInfo(TableName.valueOf(tableName), null, null, false);
+  public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd) throws IOException {
+    TableDescriptor td =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).setColumnFamily(cd).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build();
     return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), td);
   }
 
+  public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd,
+      BlockCache blockCache) throws IOException {
+    TableDescriptor td =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).setColumnFamily(cd).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build();
+    return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), td, blockCache);
+  }
+
   public void setFileSystemURI(String fsURI) {
     FS_URI = fsURI;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index ff0a88c..0e4f241 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
@@ -36,7 +37,9 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionSizeStore;
@@ -355,4 +358,14 @@ public class MockRegionServerServices implements RegionServerServices {
   public TableDescriptors getTableDescriptors() {
     return null;
   }
+
+  @Override
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.empty();
+  }
+
+  @Override
+  public Optional<MobFileCache> getMobFileCache() {
+    return Optional.empty();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
index d22772a..31c01c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
@@ -142,7 +142,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      final BlockCache cache = cacheConf.getBlockCache();
+      final BlockCache cache = cacheConf.getBlockCache().get();
       // insert data. 5 Rows are added
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -306,7 +306,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      final BlockCache cache = cacheConf.getBlockCache();
+      final BlockCache cache = cacheConf.getBlockCache().get();
       // insert data. 5 Rows are added
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index fc4c1f9..7f20195 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -196,7 +196,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       Put put = new Put(ROW);
@@ -286,7 +286,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       insertData(table);
       // flush the data
@@ -345,7 +345,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -583,7 +583,7 @@ public class TestBlockEvictionFromClient {
       HStore store = region.getStores().iterator().next();
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -647,7 +647,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -803,7 +803,7 @@ public class TestBlockEvictionFromClient {
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
       // Use the last one
-      cache = cacheConf.getBlockCache();
+      cache = cacheConf.getBlockCache().get();
     }
     return cache;
   }
@@ -830,7 +830,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       insertData(table);
@@ -896,7 +896,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       Put put = new Put(ROW);
@@ -1014,7 +1014,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       Put put = new Put(ROW);
@@ -1144,7 +1144,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
       // insert data. 2 Rows are added
       insertData(table);
       // flush the data

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 65bc3f6..e5ffd73 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
@@ -5234,14 +5234,13 @@ public class TestFromClientSide {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // establish baseline stats
       long startBlockCount = cache.getBlockCount();
       long startBlockHits = cache.getStats().getHitCount();
       long startBlockMiss = cache.getStats().getMissCount();
 
-
       // wait till baseline is stable, (minimal 500 ms)
       for (int i = 0; i < 5; i++) {
         Thread.sleep(100);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
index 11d7bb4..350a316 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
@@ -28,15 +28,16 @@ import java.util.Map;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -112,17 +113,17 @@ public class TestEncodedSeekers {
     if(includeTags) {
       testUtil.getConfiguration().setInt(HFile.FORMAT_VERSION_KEY, 3);
     }
-    CacheConfig.instantiateBlockCache(testUtil.getConfiguration());
+
     LruBlockCache cache =
-      (LruBlockCache)new CacheConfig(testUtil.getConfiguration()).getBlockCache();
-    cache.clearCache();
+        (LruBlockCache) BlockCacheFactory.createBlockCache(testUtil.getConfiguration());
     // Need to disable default row bloom filter for this test to pass.
-    HColumnDescriptor hcd = (new HColumnDescriptor(CF_NAME)).setMaxVersions(MAX_VERSIONS).
-        setDataBlockEncoding(encoding).
-        setBlocksize(BLOCK_SIZE).
-        setBloomFilterType(BloomType.NONE).
-        setCompressTags(compressTags);
-    HRegion region = testUtil.createTestRegion(TABLE_NAME, hcd);
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(CF_BYTES).setMaxVersions(MAX_VERSIONS).
+            setDataBlockEncoding(encoding).
+            setBlocksize(BLOCK_SIZE).
+            setBloomFilterType(BloomType.NONE).
+            setCompressTags(compressTags).build();
+    HRegion region = testUtil.createTestRegion(TABLE_NAME, cfd, cache);
 
     //write the data, but leave some in the memstore
     doPuts(region);
@@ -145,7 +146,6 @@ public class TestEncodedSeekers {
     assertTrue(encodingCounts.get(encodingInCache) > 0);
   }
 
-
   private void doPuts(HRegion region) throws IOException{
     LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(MIN_VALUE_SIZE, MAX_VALUE_SIZE);
      for (int i = 0; i < NUM_ROWS; ++i) {
@@ -175,7 +175,6 @@ public class TestEncodedSeekers {
     }
   }
 
-
   private void doGets(Region region) throws IOException{
     for (int i = 0; i < NUM_ROWS; ++i) {
       final byte[] rowKey = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
@@ -195,5 +194,4 @@ public class TestEncodedSeekers {
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
index 19919e0..1313f31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.Objects;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -31,7 +33,6 @@ import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.DataCacheEntry;
 import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.IndexCacheEntry;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -51,16 +52,9 @@ public class TestBlockCacheReporting {
 
   @Before
   public void setUp() throws Exception {
-    CacheConfig.clearGlobalInstances();
     this.conf = HBaseConfiguration.create();
   }
 
-  @After
-  public void tearDown() throws Exception {
-    // Let go of current block cache.
-    CacheConfig.clearGlobalInstances();
-  }
-
   private void addDataAndHits(final BlockCache bc, final int count) {
     Cacheable dce = new DataCacheEntry();
     Cacheable ice = new IndexCacheEntry();
@@ -85,39 +79,36 @@ public class TestBlockCacheReporting {
   public void testBucketCache() throws IOException {
     this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
     this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 100);
-    CacheConfig.instantiateBlockCache(this.conf);
-    CacheConfig cc = new CacheConfig(this.conf);
-    assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
-    logPerBlock(cc.getBlockCache());
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    assertTrue(blockCache instanceof CombinedBlockCache);
+    logPerBlock(blockCache);
     final int count = 3;
-    addDataAndHits(cc.getBlockCache(), count);
+    addDataAndHits(blockCache, count);
     // The below has no asserts.  It is just exercising toString and toJSON code.
-    LOG.info(Objects.toString(cc.getBlockCache().getStats()));
-    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
+    LOG.info(Objects.toString(blockCache.getStats()));
+    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(blockCache);
     LOG.info(Objects.toString(cbsbf));
     logPerFile(cbsbf);
-    bucketCacheReport(cc.getBlockCache());
+    bucketCacheReport(blockCache);
     LOG.info(BlockCacheUtil.toJSON(cbsbf));
   }
 
   @Test
   public void testLruBlockCache() throws IOException {
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    assertTrue(cc.isBlockCacheEnabled());
     assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
-    logPerBlock(cc.getBlockCache());
-    addDataAndHits(cc.getBlockCache(), 3);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    assertTrue(blockCache instanceof LruBlockCache);
+    logPerBlock(blockCache);
+    addDataAndHits(blockCache, 3);
     // The below has no asserts.  It is just exercising toString and toJSON code.
-    BlockCache bc = cc.getBlockCache();
-    LOG.info("count=" + bc.getBlockCount() + ", currentSize=" + bc.getCurrentSize() +
-        ", freeSize=" + bc.getFreeSize() );
-    LOG.info(Objects.toString(cc.getBlockCache().getStats()));
-    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
+    LOG.info("count=" + blockCache.getBlockCount() + ", currentSize=" + blockCache.getCurrentSize()
+        + ", freeSize=" + blockCache.getFreeSize());
+    LOG.info(Objects.toString(blockCache.getStats()));
+    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(blockCache);
     LOG.info(Objects.toString(cbsbf));
     logPerFile(cbsbf);
-    bucketCacheReport(cc.getBlockCache());
+    bucketCacheReport(blockCache);
     LOG.info(BlockCacheUtil.toJSON(cbsbf));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
index 7b6bbb3..4c56fff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Threads;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -165,33 +164,25 @@ public class TestCacheConfig {
 
   @Before
   public void setUp() throws Exception {
-    CacheConfig.clearGlobalInstances();
     this.conf = HBaseConfiguration.create();
   }
 
-  @After
-  public void tearDown() throws Exception {
-    // Let go of current block cache.
-    CacheConfig.clearGlobalInstances();
-  }
-
   /**
-   * @param cc
+   * @param bc The block cache instance.
+   * @param cc Cache config.
    * @param doubling If true, addition of element ups counter by 2, not 1, because element added
    * to onheap and offheap caches.
    * @param sizing True if we should run sizing test (doesn't always apply).
    */
-  void basicBlockCacheOps(final CacheConfig cc, final boolean doubling,
+  void basicBlockCacheOps(final BlockCache bc, final CacheConfig cc, final boolean doubling,
       final boolean sizing) {
-    assertTrue(cc.isBlockCacheEnabled());
     assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
-    BlockCache bc = cc.getBlockCache();
     BlockCacheKey bck = new BlockCacheKey("f", 0);
     Cacheable c = new DataCacheEntry();
     // Do asserts on block counting.
     long initialBlockCount = bc.getBlockCount();
     bc.cacheBlock(bck, c, cc.isInMemory());
-    assertEquals(doubling? 2: 1, bc.getBlockCount() - initialBlockCount);
+    assertEquals(doubling ? 2 : 1, bc.getBlockCount() - initialBlockCount);
     bc.evictBlock(bck);
     assertEquals(initialBlockCount, bc.getBlockCount());
     // Do size accounting.  Do it after the above 'warm-up' because it looks like some
@@ -209,7 +200,6 @@ public class TestCacheConfig {
   @Test
   public void testDisableCacheDataBlock() throws IOException {
     Configuration conf = HBaseConfiguration.create();
-    CacheConfig.instantiateBlockCache(conf);
     CacheConfig cacheConfig = new CacheConfig(conf);
     assertTrue(cacheConfig.shouldCacheBlockOnRead(BlockCategory.DATA));
     assertFalse(cacheConfig.shouldCacheCompressed(BlockCategory.DATA));
@@ -260,7 +250,7 @@ public class TestCacheConfig {
     HColumnDescriptor family = new HColumnDescriptor("testDisableCacheDataBlock");
     family.setBlockCacheEnabled(false);
 
-    cacheConfig = new CacheConfig(conf, family);
+    cacheConfig = new CacheConfig(conf, family, null);
     assertFalse(cacheConfig.shouldCacheBlockOnRead(BlockCategory.DATA));
     assertFalse(cacheConfig.shouldCacheCompressed(BlockCategory.DATA));
     assertFalse(cacheConfig.shouldCacheDataCompressed());
@@ -275,12 +265,11 @@ public class TestCacheConfig {
 
   @Test
   public void testCacheConfigDefaultLRUBlockCache() {
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    assertTrue(cc.isBlockCacheEnabled());
     assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
-    basicBlockCacheOps(cc, false, true);
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    basicBlockCacheOps(blockCache, cc, false, true);
+    assertTrue(blockCache instanceof LruBlockCache);
   }
 
   /**
@@ -309,18 +298,18 @@ public class TestCacheConfig {
   private void doBucketCacheConfigTest() {
     final int bcSize = 100;
     this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, bcSize);
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    basicBlockCacheOps(cc, false, false);
-    assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    basicBlockCacheOps(blockCache, cc, false, false);
+    assertTrue(blockCache instanceof CombinedBlockCache);
     // TODO: Assert sizes allocated are right and proportions.
-    CombinedBlockCache cbc = (CombinedBlockCache)cc.getBlockCache();
-    BlockCache [] bcs = cbc.getBlockCaches();
+    CombinedBlockCache cbc = (CombinedBlockCache) blockCache;
+    BlockCache[] bcs = cbc.getBlockCaches();
     assertTrue(bcs[0] instanceof LruBlockCache);
-    LruBlockCache lbc = (LruBlockCache)bcs[0];
+    LruBlockCache lbc = (LruBlockCache) bcs[0];
     assertEquals(MemorySizeUtil.getOnHeapCacheSize(this.conf), lbc.getMaxSize());
     assertTrue(bcs[1] instanceof BucketCache);
-    BucketCache bc = (BucketCache)bcs[1];
+    BucketCache bc = (BucketCache) bcs[1];
     // getMaxSize comes back in bytes but we specified size in MB
     assertEquals(bcSize, bc.getMaxSize() / (1024 * 1024));
   }
@@ -341,12 +330,12 @@ public class TestCacheConfig {
     long bcExpectedSize = 100 * 1024 * 1024; // MB.
     assertTrue(lruExpectedSize < bcExpectedSize);
     this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, bcSize);
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    basicBlockCacheOps(cc, false, false);
-    assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    basicBlockCacheOps(blockCache, cc, false, false);
+    assertTrue(blockCache instanceof CombinedBlockCache);
     // TODO: Assert sizes allocated are right and proportions.
-    CombinedBlockCache cbc = (CombinedBlockCache)cc.getBlockCache();
+    CombinedBlockCache cbc = (CombinedBlockCache) blockCache;
     LruBlockCache lbc = cbc.onHeapCache;
     assertEquals(lruExpectedSize, lbc.getMaxSize());
     BlockCache bc = cbc.l2Cache;
@@ -382,10 +371,10 @@ public class TestCacheConfig {
   public void testL2CacheWithInvalidBucketSize() {
     Configuration c = new Configuration(this.conf);
     c.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
-    c.set(CacheConfig.BUCKET_CACHE_BUCKETS_KEY, "256,512,1024,2048,4000,4096");
+    c.set(BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY, "256,512,1024,2048,4000,4096");
     c.setFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 1024);
     try {
-      CacheConfig.getBucketCache(c);
+      BlockCacheFactory.createBlockCache(c);
       fail("Should throw IllegalArgumentException when passing illegal value for bucket size");
     } catch (IllegalArgumentException e) {
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 9c2f6df..4163d55 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
@@ -31,6 +31,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -39,10 +40,11 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -51,7 +53,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -160,8 +161,7 @@ public class TestCacheOnWrite {
     Configuration conf = TEST_UTIL.getConfiguration();
     List<BlockCache> blockcaches = new ArrayList<>();
     // default
-    CacheConfig.instantiateBlockCache(conf);
-    blockcaches.add(new CacheConfig(conf).getBlockCache());
+    blockcaches.add(BlockCacheFactory.createBlockCache(conf));
 
     //set LruBlockCache.LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME to 2.0f due to HBASE-16287
     TEST_UTIL.getConfiguration().setFloat(LruBlockCache.LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME, 2.0f);
@@ -224,16 +224,16 @@ public class TestCacheOnWrite {
     conf = TEST_UTIL.getConfiguration();
     this.conf.set("dfs.datanode.data.dir.perm", "700");
     conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, INDEX_BLOCK_SIZE);
-    conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
-        BLOOM_BLOCK_SIZE);
+    conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, BLOOM_BLOCK_SIZE);
     conf.setBoolean(CacheConfig.CACHE_DATA_BLOCKS_COMPRESSED_KEY, cacheCompressedData);
     cowType.modifyConf(conf);
+    conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, cowType.shouldBeCached(BlockType.DATA));
+    conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY,
+        cowType.shouldBeCached(BlockType.LEAF_INDEX));
+    conf.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY,
+        cowType.shouldBeCached(BlockType.BLOOM_CHUNK));
+    cacheConf = new CacheConfig(conf, blockCache);
     fs = HFileSystem.get(conf);
-    cacheConf =
-        new CacheConfig(blockCache, true, true, cowType.shouldBeCached(BlockType.DATA),
-        cowType.shouldBeCached(BlockType.LEAF_INDEX),
-        cowType.shouldBeCached(BlockType.BLOOM_CHUNK), false, cacheCompressedData,
-            false, false);
   }
 
   @After
@@ -414,13 +414,11 @@ public class TestCacheOnWrite {
     final String cf = "myCF";
     final byte[] cfBytes = Bytes.toBytes(cf);
     final int maxVersions = 3;
-    HRegion region = TEST_UTIL.createTestRegion(table,
-        new HColumnDescriptor(cf)
-            .setCompressionType(compress)
-            .setBloomFilterType(BLOOM_TYPE)
-            .setMaxVersions(maxVersions)
-            .setDataBlockEncoding(NoOpDataBlockEncoder.INSTANCE.getDataBlockEncoding())
-    );
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(cfBytes).setCompressionType(compress)
+            .setBloomFilterType(BLOOM_TYPE).setMaxVersions(maxVersions)
+            .setDataBlockEncoding(NoOpDataBlockEncoder.INSTANCE.getDataBlockEncoding()).build();
+    HRegion region = TEST_UTIL.createTestRegion(table, cfd, blockCache);
     int rowIdx = 0;
     long ts = EnvironmentEdgeManager.currentTime();
     for (int iFile = 0; iFile < 5; ++iFile) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
index 5612c1b..fd39f48 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
@@ -25,14 +25,14 @@ import java.util.Arrays;
 import java.util.Collection;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -103,24 +103,20 @@ public class TestForceCacheImportantBlocks {
 
   @Before
   public void setup() {
-    // Make sure we make a new one each time.
-    CacheConfig.clearGlobalInstances();
     HFile.DATABLOCK_READ_COUNT.reset();
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
   }
 
   @Test
   public void testCacheBlocks() throws IOException {
     // Set index block size to be the same as normal block size.
     TEST_UTIL.getConfiguration().setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, BLOCK_SIZE);
-    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(CF)).setMaxVersions(MAX_VERSIONS).
-      setCompressionType(COMPRESSION_ALGORITHM).
-      setBloomFilterType(BLOOM_TYPE);
-    hcd.setBlocksize(BLOCK_SIZE);
-    hcd.setBlockCacheEnabled(cfCacheEnabled);
-    HRegion region = TEST_UTIL.createTestRegion(TABLE, hcd);
-    BlockCache cache = region.getStore(hcd.getName()).getCacheConfig().getBlockCache();
-    CacheStats stats = cache.getStats();
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(TEST_UTIL.getConfiguration());
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CF)).setMaxVersions(MAX_VERSIONS)
+            .setCompressionType(COMPRESSION_ALGORITHM).setBloomFilterType(BLOOM_TYPE)
+            .setBlocksize(BLOCK_SIZE).setBlockCacheEnabled(cfCacheEnabled).build();
+    HRegion region = TEST_UTIL.createTestRegion(TABLE, cfd, blockCache);
+    CacheStats stats = blockCache.getStats();
     writeTestData(region);
     assertEquals(0, stats.getHitCount());
     assertEquals(0, HFile.DATABLOCK_READ_COUNT.sum());

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 7053fce..2a613de 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
@@ -86,13 +86,14 @@ public class TestHFile  {
     TEST_UTIL.getDataTestDir("TestHFile").toString();
   private final int minBlockSize = 512;
   private static String localFormatter = "%010d";
-  private static CacheConfig cacheConf = null;
+  private static CacheConfig cacheConf;
   private static Configuration conf ;
   private static FileSystem fs;
 
   @BeforeClass
   public static void setUp() throws Exception {
     conf = TEST_UTIL.getConfiguration();
+    cacheConf = new CacheConfig(conf);
     fs = TEST_UTIL.getTestFileSystem();
   }
 
@@ -162,7 +163,6 @@ public class TestHFile  {
    */
   @Test
   public void testEmptyHFile() throws IOException {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path f = new Path(ROOT_DIR, testName.getMethodName());
     HFileContext context = new HFileContextBuilder().withIncludesTags(false).build();
     Writer w =
@@ -179,7 +179,6 @@ public class TestHFile  {
    */
   @Test
   public void testCorrupt0LengthHFile() throws IOException {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path f = new Path(ROOT_DIR, testName.getMethodName());
     FSDataOutputStream fsos = fs.create(f);
     fsos.close();
@@ -213,7 +212,6 @@ public class TestHFile  {
    */
   @Test
   public void testCorruptTruncatedHFile() throws IOException {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path f = new Path(ROOT_DIR, testName.getMethodName());
     HFileContext  context = new HFileContextBuilder().build();
     Writer w = HFile.getWriterFactory(conf, cacheConf).withPath(this.fs, f)
@@ -315,7 +313,6 @@ public class TestHFile  {
     if (useTags) {
       conf.setInt("hfile.format.version", 3);
     }
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path  ncHFile = new Path(ROOT_DIR, "basic.hfile." + codec.toString() + useTags);
     FSDataOutputStream fout = createFSOutput(ncHFile);
     HFileContext meta = new HFileContextBuilder()
@@ -411,7 +408,6 @@ public class TestHFile  {
   }
 
   private void metablocks(final String compress) throws Exception {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path mFile = new Path(ROOT_DIR, "meta.hfile");
     FSDataOutputStream fout = createFSOutput(mFile);
     HFileContext meta = new HFileContextBuilder()
@@ -445,7 +441,6 @@ public class TestHFile  {
 
   @Test
   public void testNullMetaBlocks() throws Exception {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     for (Compression.Algorithm compressAlgo :
         HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS) {
       Path mFile = new Path(ROOT_DIR, "nometa_" + compressAlgo + ".hfile");

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index a588341..48080b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -32,6 +32,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -246,7 +247,7 @@ public class TestHFileBlock {
   @Test
   public void testNoCompression() throws IOException {
     CacheConfig cacheConf = Mockito.mock(CacheConfig.class);
-    Mockito.when(cacheConf.isBlockCacheEnabled()).thenReturn(false);
+    Mockito.when(cacheConf.getBlockCache()).thenReturn(Optional.empty());
 
     HFileBlock block =
       createTestV2Block(NONE, includesMemstoreTS, false).getBlockForCaching(cacheConf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 efe76aa..78f8584 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
@@ -530,9 +530,8 @@ public class TestHFileBlockIndex {
     conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize);
     // should open hfile.block.index.cacheonwrite
     conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, true);
-    CacheConfig.instantiateBlockCache(conf);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    BlockCache blockCache = cacheConf.getBlockCache();
+    CacheConfig cacheConf = new CacheConfig(conf, BlockCacheFactory.createBlockCache(conf));
+    BlockCache blockCache = cacheConf.getBlockCache().get();
     // Evict all blocks that were cached-on-write by the previous invocation.
     blockCache.evictBlocksByHfileName(hfilePath.getName());
     // Write the HFile
@@ -589,9 +588,8 @@ public class TestHFileBlockIndex {
   public void testHFileWriterAndReader() throws IOException {
     Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
         "hfile_for_block_index");
-    CacheConfig.instantiateBlockCache(conf);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    BlockCache blockCache = cacheConf.getBlockCache();
+    CacheConfig cacheConf = new CacheConfig(conf, BlockCacheFactory.createBlockCache(conf));
+    BlockCache blockCache = cacheConf.getBlockCache().get();
 
     for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; ++testI) {
       int indexBlockSize = INDEX_CHUNK_SIZES[testI];

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
index 4542a3c..5935f91 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
@@ -82,13 +82,11 @@ public class TestLazyDataBlockDecompression {
 
   @Before
   public void setUp() throws IOException {
-    CacheConfig.clearGlobalInstances();
     fs = FileSystem.get(TEST_UTIL.getConfiguration());
   }
 
   @After
   public void tearDown() {
-    CacheConfig.clearGlobalInstances();
     fs = null;
   }
 
@@ -159,12 +157,11 @@ public class TestLazyDataBlockDecompression {
     lazyCompressDisabled.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressDisabled.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressDisabled.setBoolean(CacheConfig.CACHE_DATA_BLOCKS_COMPRESSED_KEY, false);
-    CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE =
-      new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressDisabled);
-    CacheConfig cc = new CacheConfig(lazyCompressDisabled);
+    CacheConfig cc = new CacheConfig(lazyCompressDisabled,
+        new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressDisabled));
     assertFalse(cc.shouldCacheDataCompressed());
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
-    LruBlockCache disabledBlockCache = (LruBlockCache) cc.getBlockCache();
+    assertTrue(cc.getBlockCache().get() instanceof LruBlockCache);
+    LruBlockCache disabledBlockCache = (LruBlockCache) cc.getBlockCache().get();
     LOG.info("disabledBlockCache=" + disabledBlockCache);
     assertEquals("test inconsistency detected.", maxSize, disabledBlockCache.getMaxSize());
     assertTrue("eviction thread spawned unintentionally.",
@@ -194,12 +191,11 @@ public class TestLazyDataBlockDecompression {
     lazyCompressEnabled.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressEnabled.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressEnabled.setBoolean(CacheConfig.CACHE_DATA_BLOCKS_COMPRESSED_KEY, true);
-    CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE =
-      new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressEnabled);
-    cc = new CacheConfig(lazyCompressEnabled);
+    cc = new CacheConfig(lazyCompressEnabled,
+        new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressEnabled));
     assertTrue("test improperly configured.", cc.shouldCacheDataCompressed());
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
-    LruBlockCache enabledBlockCache = (LruBlockCache) cc.getBlockCache();
+    assertTrue(cc.getBlockCache().get() instanceof LruBlockCache);
+    LruBlockCache enabledBlockCache = (LruBlockCache) cc.getBlockCache().get();
     LOG.info("enabledBlockCache=" + enabledBlockCache);
     assertEquals("test inconsistency detected", maxSize, enabledBlockCache.getMaxSize());
     assertTrue("eviction thread spawned unintentionally.",

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 811df14..9986bba 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
@@ -17,10 +17,12 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Random;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -28,8 +30,9 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.testclassification.IOTests;
@@ -57,24 +60,25 @@ public class TestPrefetch {
   private Configuration conf;
   private CacheConfig cacheConf;
   private FileSystem fs;
+  private BlockCache blockCache;
 
   @Before
   public void setUp() throws IOException {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true);
     fs = HFileSystem.get(conf);
-    CacheConfig.blockCacheDisabled = false;
-    CacheConfig.instantiateBlockCache(conf);
-    cacheConf = new CacheConfig(conf);
+    blockCache = BlockCacheFactory.createBlockCache(conf);
+    cacheConf = new CacheConfig(conf, blockCache);
   }
 
   @Test
   public void testPrefetchSetInHCDWorks() {
-    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("f"));
-    hcd.setPrefetchBlocksOnOpen(true);
+    ColumnFamilyDescriptor columnFamilyDescriptor =
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f")).setPrefetchBlocksOnOpen(true)
+            .build();
     Configuration c = HBaseConfiguration.create();
     assertFalse(c.getBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, false));
-    CacheConfig cc = new CacheConfig(c, hcd);
+    CacheConfig cc = new CacheConfig(c, columnFamilyDescriptor, blockCache);
     assertTrue(cc.shouldPrefetchOnOpen());
   }
 
@@ -119,7 +123,7 @@ public class TestPrefetch {
     }
 
     // Check that all of the data blocks were preloaded
-    BlockCache blockCache = cacheConf.getBlockCache();
+    BlockCache blockCache = cacheConf.getBlockCache().get();
     long offset = 0;
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
       HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 18e8e70..ad6a0ab 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
@@ -25,29 +25,28 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-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;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -88,7 +87,6 @@ public class TestScannerFromBucketCache {
       conf.setFloat("hbase.regionserver.global.memstore.size", 0.1f);
     }
     tableName = TableName.valueOf(name.getMethodName());
-    CacheConfig.instantiateBlockCache(conf);
   }
 
   @After
@@ -96,7 +94,6 @@ public class TestScannerFromBucketCache {
     EnvironmentEdgeManagerTestHelper.reset();
     LOG.info("Cleaning test directory: " + test_util.getDataTestDir());
     test_util.cleanupTestDir();
-    CacheConfig.clearGlobalInstances();
   }
 
   String getName() {
@@ -210,9 +207,7 @@ public class TestScannerFromBucketCache {
       Thread.sleep(500);
       // do the scan again and verify. This time it should be from the bucket cache in offheap mode
       // but one of the cell will be copied due to the asSubByteBuff call
-      Scan scan = new Scan(row1);
-      scan.addFamily(fam1);
-      scan.setMaxVersions(10);
+      Scan scan = new Scan().withStartRow(row1).addFamily(fam1).readVersions(10);
       actual = new ArrayList<>();
       InternalScanner scanner = region.getScanner(scan);
 
@@ -290,9 +285,7 @@ public class TestScannerFromBucketCache {
   }
 
   private List<Cell> performScan(byte[] row1, byte[] fam1) throws IOException {
-    Scan scan = new Scan(row1);
-    scan.addFamily(fam1);
-    scan.setMaxVersions(MAX_VERSIONS);
+    Scan scan = new Scan().withStartRow(row1).addFamily(fam1).readVersions(MAX_VERSIONS);
     List<Cell> actual = new ArrayList<>();
     InternalScanner scanner = region.getScanner(scan);
 
@@ -307,32 +300,19 @@ public class TestScannerFromBucketCache {
   }
 
   private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
-      String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
+      String callingMethod, Configuration conf, HBaseTestingUtility testUtil, 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);
-    return initHRegion(tableName, startKey, stopKey, callingMethod, conf, test_util, isReadOnly,
-      Durability.SYNC_WAL, wal, families);
-  }
-
-  /**
-   * @param tableName
-   * @param startKey
-   * @param stopKey
-   * @param callingMethod
-   * @param conf
-   * @param isReadOnly
-   * @param families
-   * @throws IOException
-   * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
-   *         when done.
-   */
-  private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
-      String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
-      Durability durability, WAL wal, byte[]... families) throws IOException {
-    return test_util.createLocalHRegion(tableName, startKey, stopKey, isReadOnly, durability, wal,
-      families);
+    RegionInfo regionInfo =
+        RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).setEndKey(stopKey).build();
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    builder.setReadOnly(isReadOnly).setDurability(Durability.SYNC_WAL);
+    for (byte[] family : families) {
+      builder.setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(Integer.MAX_VALUE)
+              .build());
+    }
+    return HBaseTestingUtility
+        .createRegionAndWAL(regionInfo, testUtil.getDataTestDir(callingMethod), conf,
+            builder.build(), BlockCacheFactory.createBlockCache(conf));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
index d27b041..ed440e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
@@ -123,10 +123,7 @@ public class TestScannerSelectionUsingKeyRange {
     }
 
     Scan scan = new Scan(Bytes.toBytes("aaa"), Bytes.toBytes("aaz"));
-    CacheConfig.blockCacheDisabled = false;
-    CacheConfig.instantiateBlockCache(conf);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    LruBlockCache cache = (LruBlockCache) cacheConf.getBlockCache();
+    LruBlockCache cache = (LruBlockCache) BlockCacheFactory.createBlockCache(conf);
     cache.clearCache();
     InternalScanner scanner = region.getScanner(scan);
     List<Cell> results = new ArrayList<>();


[26/47] hbase git commit: HBASE-21621 Reversed scan does not return expected number of rows

Posted by zh...@apache.org.
HBASE-21621 Reversed scan does not return expected number of rows

The unit test is contributed by Nihal Jain


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

Branch: refs/heads/HBASE-21512
Commit: 7c0a3cc265f1351363dc88f2f70855b3273dd8c6
Parents: b2bf22e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Dec 20 12:34:34 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Sun Dec 23 16:19:05 2018 +0800

----------------------------------------------------------------------
 .../regionserver/ReversedStoreScanner.java      |  5 +--
 .../hadoop/hbase/regionserver/StoreScanner.java | 10 ++++-
 .../hbase/client/TestFromClientSide3.java       |  2 +-
 .../client/TestScannersFromClientSide.java      | 43 ++++++++++++++++++++
 .../hbase/regionserver/TestStoreScanner.java    |  8 +++-
 5 files changed, 61 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7c0a3cc2/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 90e1129..491e6ef 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
@@ -59,10 +59,9 @@ public class ReversedStoreScanner extends StoreScanner implements KeyValueScanne
   }
 
   @Override
-  protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
+  protected KeyValueHeap newKVHeap(List<? extends KeyValueScanner> scanners,
       CellComparator comparator) throws IOException {
-    // Combine all seeked scanners with a heap
-    heap = new ReversedKeyValueHeap(scanners, comparator);
+    return new ReversedKeyValueHeap(scanners, comparator);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c0a3cc2/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 b318950..d777706 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
@@ -403,10 +403,16 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     }
   }
 
+  @VisibleForTesting
   protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
       CellComparator comparator) throws IOException {
     // Combine all seeked scanners with a heap
-    heap = new KeyValueHeap(scanners, comparator);
+    heap = newKVHeap(scanners, comparator);
+  }
+
+  protected KeyValueHeap newKVHeap(List<? extends KeyValueScanner> scanners,
+      CellComparator comparator) throws IOException {
+    return new KeyValueHeap(scanners, comparator);
   }
 
   /**
@@ -1037,7 +1043,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       newCurrentScanners = new ArrayList<>(fileScanners.size() + memstoreScanners.size());
       newCurrentScanners.addAll(fileScanners);
       newCurrentScanners.addAll(memstoreScanners);
-      newHeap = new KeyValueHeap(newCurrentScanners, comparator);
+      newHeap = newKVHeap(newCurrentScanners, comparator);
     } catch (Exception e) {
       LOG.warn("failed to switch to stream read", e);
       if (fileScanners != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c0a3cc2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index cbfa1bf..1315d4a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -1059,7 +1059,7 @@ public class TestFromClientSide3 {
     }
   }
 
-  private static byte[] generateHugeValue(int size) {
+  static byte[] generateHugeValue(int size) {
     Random rand = ThreadLocalRandom.current();
     byte[] value = new byte[size];
     for (int i = 0; i < value.length; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c0a3cc2/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 b91e205..af02482 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.HConstants.RPC_CODEC_CONF_KEY;
+import static org.apache.hadoop.hbase.client.TestFromClientSide3.generateHugeValue;
 import static org.apache.hadoop.hbase.ipc.RpcClient.DEFAULT_CODEC_CLASS;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -935,4 +936,46 @@ public class TestScannersFromClientSide {
       }
     }
   }
+
+  @Test
+  public void testReverseScanWithFlush() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    final int BATCH_SIZE = 10;
+    final int ROWS_TO_INSERT = 100;
+    final byte[] LARGE_VALUE = generateHugeValue(128 * 1024);
+
+    try (Table table = TEST_UTIL.createTable(tableName, FAMILY);
+        Admin admin = TEST_UTIL.getAdmin()) {
+      List<Put> putList = new ArrayList<>();
+      for (long i = 0; i < ROWS_TO_INSERT; i++) {
+        Put put = new Put(Bytes.toBytes(i));
+        put.addColumn(FAMILY, QUALIFIER, LARGE_VALUE);
+        putList.add(put);
+
+        if (putList.size() >= BATCH_SIZE) {
+          table.put(putList);
+          admin.flush(tableName);
+          putList.clear();
+        }
+      }
+
+      if (!putList.isEmpty()) {
+        table.put(putList);
+        admin.flush(tableName);
+        putList.clear();
+      }
+
+      Scan scan = new Scan();
+      scan.setReversed(true);
+      int count = 0;
+
+      try (ResultScanner results = table.getScanner(scan)) {
+        for (Result result : results) {
+          count++;
+        }
+      }
+      assertEquals("Expected " + ROWS_TO_INSERT + " rows in the table but it is " + count,
+          ROWS_TO_INSERT, count);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c0a3cc2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index db50a85..687b780 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@ -170,7 +170,13 @@ public class TestStoreScanner {
       if (count == null) {
         count = new AtomicInteger(0);
       }
-      heap = new KeyValueHeapWithCount(scanners, comparator, count);
+      heap = newKVHeap(scanners, comparator);
+    }
+
+    @Override
+    protected KeyValueHeap newKVHeap(List<? extends KeyValueScanner> scanners,
+        CellComparator comparator) throws IOException {
+      return new KeyValueHeapWithCount(scanners, comparator, count);
     }
 
     @Override


[11/47] hbase git commit: HBASE-21590 Optimize trySkipToNextColumn in StoreScanner a bit. (addendum)

Posted by zh...@apache.org.
HBASE-21590 Optimize trySkipToNextColumn in StoreScanner a bit. (addendum)


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

Branch: refs/heads/HBASE-21512
Commit: 491153488ee5b19de22fd72e55dd5039399bb727
Parents: 2b003c5
Author: Sean Busbey <bu...@apache.org>
Authored: Fri Dec 14 11:23:36 2018 -0600
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Dec 14 17:08:22 2018 -0600

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/StoreScanner.java    | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/49115348/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 e7a4528..91ca592 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
@@ -807,8 +807,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     Cell previousIndexedKey = null;
     do {
       Cell nextIndexedKey = getNextIndexedKey();
-      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && (nextIndexedKey == previousIndexedKey || matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0)) {
+      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&
+          (nextIndexedKey == previousIndexedKey ||
+          matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0)) {
         this.heap.next();
         ++kvsScanned;
         previousIndexedKey = nextIndexedKey;
@@ -832,8 +833,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     Cell previousIndexedKey = null;
     do {
       Cell nextIndexedKey = getNextIndexedKey();
-      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && (nextIndexedKey == previousIndexedKey || matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0)) {
+      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&
+          (nextIndexedKey == previousIndexedKey ||
+          matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0)) {
         this.heap.next();
         ++kvsScanned;
         previousIndexedKey = nextIndexedKey;


[29/47] hbase git commit: HBASE-21545 NEW_VERSION_BEHAVIOR breaks Get/Scan with specified columns

Posted by zh...@apache.org.
HBASE-21545 NEW_VERSION_BEHAVIOR breaks Get/Scan with specified columns

Signed-off-by: Duo Zhang <zh...@apache.org>
Signed-off-by: stack <st...@apache.org>
Signed-off-by: Sakthi


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

Branch: refs/heads/HBASE-21512
Commit: dbafa1be83d6f5894f1cc3eadb07ae0c3096de3a
Parents: 59f77de
Author: Andrey Elenskiy <an...@arista.com>
Authored: Tue Dec 4 12:10:38 2018 -0800
Committer: stack <st...@apache.org>
Committed: Sun Dec 23 22:01:11 2018 -0800

----------------------------------------------------------------------
 .../querymatcher/NewVersionBehaviorTracker.java |  39 +++----
 .../hadoop/hbase/HBaseTestingUtility.java       |  58 ++++++++--
 ...estGetScanColumnsWithNewVersionBehavior.java | 109 +++++++++++++++++++
 .../TestNewVersionBehaviorTracker.java          |  36 ++++++
 4 files changed, 214 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dbafa1be/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NewVersionBehaviorTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NewVersionBehaviorTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NewVersionBehaviorTracker.java
index 4027766..16ac84c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NewVersionBehaviorTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/NewVersionBehaviorTracker.java
@@ -277,26 +277,26 @@ public class NewVersionBehaviorTracker implements ColumnTracker, DeleteTracker {
 
   @Override
   public MatchCode checkColumn(Cell cell, byte type) throws IOException {
-    if (done()) {
-      // No more columns left, we are done with this query
-      return ScanQueryMatcher.MatchCode.SEEK_NEXT_ROW; // done_row
+    if (columns == null) {
+        return MatchCode.INCLUDE;
     }
-    if (columns != null) {
-      while (columnIndex < columns.length) {
-        int c = Bytes.compareTo(columns[columnIndex], 0, columns[columnIndex].length,
-            cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
-        if (c < 0) {
-          columnIndex++;
-        } else if (c == 0) {
-          // We drop old version in #isDeleted, so here we must return INCLUDE.
-          return MatchCode.INCLUDE;
-        } else {
-          return MatchCode.SEEK_NEXT_COL;
-        }
+
+    while (!done()) {
+      int c = CellUtil.compareQualifiers(cell,
+        columns[columnIndex], 0, columns[columnIndex].length);
+      if (c < 0) {
+        return MatchCode.SEEK_NEXT_COL;
       }
-      return MatchCode.SEEK_NEXT_ROW;
+
+      if (c == 0) {
+        // We drop old version in #isDeleted, so here we must return INCLUDE.
+        return MatchCode.INCLUDE;
+      }
+
+      columnIndex++;
     }
-    return MatchCode.INCLUDE;
+    // No more columns left, we are done with this query
+    return MatchCode.SEEK_NEXT_ROW;
   }
 
   @Override
@@ -351,10 +351,7 @@ public class NewVersionBehaviorTracker implements ColumnTracker, DeleteTracker {
 
   @Override
   public boolean done() {
-    // lastCq* have been updated to this cell.
-    return !(columns == null || lastCqArray == null) && Bytes
-        .compareTo(lastCqArray, lastCqOffset, lastCqLength, columns[columnIndex], 0,
-            columns[columnIndex].length) > 0;
+    return columns != null && columnIndex >= columns.length;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/dbafa1be/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 7bfbfe1..796dbc3 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
@@ -713,6 +713,18 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       new Path(root, "mapreduce-am-staging-root-dir").toString());
   }
 
+  /**
+   *  Check whether the tests should assume NEW_VERSION_BEHAVIOR when creating
+   *  new column families. Default to false.
+   */
+  public boolean isNewVersionBehaviorEnabled(){
+    final String propName = "hbase.tests.new.version.behavior";
+    String v = System.getProperty(propName);
+    if (v != null){
+      return Boolean.parseBoolean(v);
+    }
+    return false;
+  }
 
   /**
    *  Get the HBase setting for dfs.client.read.shortcircuit from the conf or a system property.
@@ -1576,9 +1588,13 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       BloomType type, int blockSize, Configuration c) throws IOException {
     TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(htd);
     for (byte[] family : families) {
-      builder.setColumnFamily(
-          ColumnFamilyDescriptorBuilder.newBuilder(family).setBloomFilterType(type)
-              .setBlocksize(blockSize).build());
+      ColumnFamilyDescriptorBuilder cfdb = ColumnFamilyDescriptorBuilder.newBuilder(family)
+        .setBloomFilterType(type)
+        .setBlocksize(blockSize);
+      if (isNewVersionBehaviorEnabled()) {
+          cfdb.setNewVersionBehavior(true);
+      }
+      builder.setColumnFamily(cfdb.build());
     }
     TableDescriptor td = builder.build();
     getAdmin().createTable(td, splitKeys);
@@ -1597,7 +1613,14 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    */
   public Table createTable(TableDescriptor htd, byte[][] splitRows)
       throws IOException {
-    getAdmin().createTable(htd, splitRows);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(htd);
+    if (isNewVersionBehaviorEnabled()) {
+      for (ColumnFamilyDescriptor family : htd.getColumnFamilies()) {
+         builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family)
+           .setNewVersionBehavior(true).build());
+      }
+    }
+    getAdmin().createTable(builder.build(), splitRows);
     // HBaseAdmin only waits for regions to appear in hbase:meta
     // we should wait until they are assigned
     waitUntilAllRegionsAssigned(htd.getTableName());
@@ -1661,6 +1684,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     for (byte[] family : families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions);
+      if (isNewVersionBehaviorEnabled()) {
+        hcd.setNewVersionBehavior(true);
+      }
       desc.addFamily(hcd);
     }
     getAdmin().createTable(desc, splitKeys);
@@ -1699,6 +1725,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       HColumnDescriptor hcd = new HColumnDescriptor(family)
           .setMaxVersions(numVersions)
           .setBlocksize(blockSize);
+      if (isNewVersionBehaviorEnabled()) {
+        hcd.setNewVersionBehavior(true);
+      }
       desc.addFamily(hcd);
     }
     getAdmin().createTable(desc);
@@ -1715,6 +1744,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
         HColumnDescriptor hcd = new HColumnDescriptor(family)
             .setMaxVersions(numVersions)
             .setBlocksize(blockSize);
+        if (isNewVersionBehaviorEnabled()) {
+          hcd.setNewVersionBehavior(true);
+        }
         desc.addFamily(hcd);
       }
       if(cpName != null) {
@@ -1743,6 +1775,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     for (byte[] family : families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family)
           .setMaxVersions(numVersions[i]);
+      if (isNewVersionBehaviorEnabled()) {
+        hcd.setNewVersionBehavior(true);
+      }
       desc.addFamily(hcd);
       i++;
     }
@@ -1765,6 +1800,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor(family);
+    if (isNewVersionBehaviorEnabled()) {
+      hcd.setNewVersionBehavior(true);
+    }
     desc.addFamily(hcd);
     getAdmin().createTable(desc, splitRows);
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
@@ -1886,13 +1924,16 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
     HTableDescriptor htd = new HTableDescriptor(name);
     for (byte[] cfName : new byte[][]{ fam1, fam2, fam3 }) {
-      htd.addFamily(new HColumnDescriptor(cfName)
+      HColumnDescriptor hcd = new HColumnDescriptor(cfName)
           .setMinVersions(minVersions)
           .setMaxVersions(versions)
           .setKeepDeletedCells(keepDeleted)
           .setBlockCacheEnabled(false)
-          .setTimeToLive(ttl)
-      );
+          .setTimeToLive(ttl);
+      if (isNewVersionBehaviorEnabled()) {
+          hcd.setNewVersionBehavior(true);
+      }
+      htd.addFamily(hcd);
     }
     return htd;
   }
@@ -1918,6 +1959,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     for (byte[] family : families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family)
           .setMaxVersions(maxVersions);
+      if (isNewVersionBehaviorEnabled()) {
+          hcd.setNewVersionBehavior(true);
+      }
       desc.addFamily(hcd);
     }
     return desc;

http://git-wip-us.apache.org/repos/asf/hbase/blob/dbafa1be/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanColumnsWithNewVersionBehavior.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanColumnsWithNewVersionBehavior.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanColumnsWithNewVersionBehavior.java
new file mode 100644
index 0000000..7eb3b35
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGetScanColumnsWithNewVersionBehavior.java
@@ -0,0 +1,109 @@
+/**
+ * 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.assertArrayEquals;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+import java.util.ArrayList;
+
+/**
+ * Testcase for HBASE-21032, where use the wrong readType from a Scan instance which is actually a
+ * get scan and cause returning only 1 cell per rpc call.
+ */
+@Category({ ClientTests.class, MediumTests.class })
+public class TestGetScanColumnsWithNewVersionBehavior {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestGetScanColumnsWithNewVersionBehavior.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final TableName TABLE = TableName.valueOf("table");
+  private static final byte[] CF = { 'c', 'f' };
+  private static final byte[] ROW = { 'r', 'o', 'w' };
+  private static final byte[] COLA = { 'a' };
+  private static final byte[] COLB = { 'b' };
+  private static final byte[] COLC = { 'c' };
+  private static final long TS = 42;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.startMiniCluster(1);
+    ColumnFamilyDescriptor cd = ColumnFamilyDescriptorBuilder
+        .newBuilder(CF)
+        .setNewVersionBehavior(true)
+        .build();
+    TEST_UTIL.createTable(TableDescriptorBuilder
+        .newBuilder(TABLE)
+        .setColumnFamily(cd)
+        .build(), null);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws IOException {
+    try (Table t = TEST_UTIL.getConnection().getTable(TABLE)) {
+      Cell [] expected = new Cell[2];
+      expected[0] = new KeyValue(ROW, CF, COLA, TS, COLA);
+      expected[1] = new KeyValue(ROW, CF, COLC, TS, COLC);
+
+      Put p = new Put(ROW);
+      p.addColumn(CF, COLA, TS, COLA);
+      p.addColumn(CF, COLB, TS, COLB);
+      p.addColumn(CF, COLC, TS, COLC);
+      t.put(p);
+
+      // check get request
+      Get get = new Get(ROW);
+      get.addColumn(CF, COLA);
+      get.addColumn(CF, COLC);
+      Result getResult = t.get(get);
+      assertArrayEquals(expected, getResult.rawCells());
+
+      // check scan request
+      Scan scan = new Scan(ROW);
+      scan.addColumn(CF, COLA);
+      scan.addColumn(CF, COLC);
+      ResultScanner scanner = t.getScanner(scan);
+      List scanResult = new ArrayList<Cell>();
+      for (Result result = scanner.next(); (result != null); result = scanner.next()) {
+          scanResult.addAll(result.listCells());
+      }
+      assertArrayEquals(expected, scanResult.toArray(new Cell[scanResult.size()]));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/dbafa1be/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java
index 098c5ff..d3542eb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestNewVersionBehaviorTracker.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver.querymatcher;
 
+import java.util.TreeSet;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
@@ -33,6 +34,7 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+
 @Category({ RegionServerTests.class, SmallTests.class })
 public class TestNewVersionBehaviorTracker {
 
@@ -40,12 +42,46 @@ public class TestNewVersionBehaviorTracker {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestNewVersionBehaviorTracker.class);
 
+  private final byte[] col0 = Bytes.toBytes("col0");
   private final byte[] col1 = Bytes.toBytes("col1");
   private final byte[] col2 = Bytes.toBytes("col2");
+  private final byte[] col3 = Bytes.toBytes("col3");
+  private final byte[] col4 = Bytes.toBytes("col4");
   private final byte[] row = Bytes.toBytes("row");
   private final byte[] family = Bytes.toBytes("family");
   private final byte[] value = Bytes.toBytes("value");
   private final CellComparator comparator = CellComparatorImpl.COMPARATOR;
+
+  @Test
+  public void testColumns() throws IOException {
+    TreeSet<byte[]> trackedColumns = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+    trackedColumns.add(col1);
+    trackedColumns.add(col3);
+
+    NewVersionBehaviorTracker tracker =
+        new NewVersionBehaviorTracker(trackedColumns, comparator, 1, 3, 3, 10000);
+
+    KeyValue keyValue = new KeyValue(row, family, col0, 20000, KeyValue.Type.Put, value);
+    assertEquals(DeleteResult.NOT_DELETED, tracker.isDeleted(keyValue));
+    assertEquals(MatchCode.SEEK_NEXT_COL, tracker.checkColumn(keyValue, keyValue.getTypeByte()));
+
+    keyValue = new KeyValue(row, family, col1, 20000, KeyValue.Type.Put, value);
+    assertEquals(DeleteResult.NOT_DELETED, tracker.isDeleted(keyValue));
+    assertEquals(MatchCode.INCLUDE, tracker.checkColumn(keyValue, keyValue.getTypeByte()));
+
+    keyValue = new KeyValue(row, family, col2, 20000, KeyValue.Type.Put, value);
+    assertEquals(DeleteResult.NOT_DELETED, tracker.isDeleted(keyValue));
+    assertEquals(MatchCode.SEEK_NEXT_COL, tracker.checkColumn(keyValue, keyValue.getTypeByte()));
+
+    keyValue = new KeyValue(row, family, col3, 20000, KeyValue.Type.Put, value);
+    assertEquals(DeleteResult.NOT_DELETED, tracker.isDeleted(keyValue));
+    assertEquals(MatchCode.INCLUDE, tracker.checkColumn(keyValue, keyValue.getTypeByte()));
+
+    keyValue = new KeyValue(row, family, col4, 20000, KeyValue.Type.Put, value);
+    assertEquals(DeleteResult.NOT_DELETED, tracker.isDeleted(keyValue));
+    assertEquals(MatchCode.SEEK_NEXT_ROW, tracker.checkColumn(keyValue, keyValue.getTypeByte()));
+  }
+
   @Test
   public void testMaxVersionMask() {
     NewVersionBehaviorTracker tracker =


[44/47] hbase git commit: HBASE-21646 Flakey TestTableSnapshotInputFormat; DisableTable not completing... Amendment to fix checkstyle complaint

Posted by zh...@apache.org.
HBASE-21646 Flakey TestTableSnapshotInputFormat; DisableTable not completing... Amendment to fix checkstyle complaint

Includes fix for checkstyle complaint.


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

Branch: refs/heads/HBASE-21512
Commit: 7755d4beeddfee9b72446ccd18d7918278eecc83
Parents: b620334
Author: stack <st...@apache.org>
Authored: Fri Dec 28 14:42:22 2018 -0800
Committer: stack <st...@apache.org>
Committed: Fri Dec 28 14:48:23 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/mapred/TestTableSnapshotInputFormat.java     | 7 +++++++
 .../hbase/mapreduce/TableSnapshotInputFormatTestBase.java     | 5 -----
 .../hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java  | 5 +++++
 3 files changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7755d4be/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
index b61ed07..c591af6 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.lib.NullOutputFormat;
 import org.junit.Assert;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -310,4 +311,10 @@ public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBa
       }
     }
   }
+
+  @Ignore // Ignored in mapred package because it keeps failing but allowed in mapreduce package.
+  @Test
+  public void testWithMapReduceMultipleMappersPerRegion() throws Exception {
+    testWithMapReduce(UTIL, "testWithMapReduceMultiRegion", 10, 5, 50, false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7755d4be/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java
index 744c356..5e7ea7a 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java
@@ -108,11 +108,6 @@ public abstract class TableSnapshotInputFormatTestBase {
   }
 
   @Test
-  public void testWithMapReduceMultipleMappersPerRegion() throws Exception {
-    testWithMapReduce(UTIL, "testWithMapReduceMultiRegion", 10, 5, 50, false);
-  }
-
-  @Test
   // run the MR job while HBase is offline
   public void testWithMapReduceAndOfflineHBaseMultiRegion() throws Exception {
     testWithMapReduce(UTIL, "testWithMapReduceAndOfflineHBaseMultiRegion", 10, 1, 8, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7755d4be/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
index f61c222..358af24 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
@@ -473,4 +473,9 @@ public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBa
       }
     }
   }
+
+  @Test
+  public void testWithMapReduceMultipleMappersPerRegion() throws Exception {
+    testWithMapReduce(UTIL, "testWithMapReduceMultiRegion", 10, 5, 50, false);
+  }
 }


[19/47] hbase git commit: HBASE-21535, Zombie Master detector is not working

Posted by zh...@apache.org.
HBASE-21535, Zombie Master detector is not working


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

Branch: refs/heads/HBASE-21512
Commit: fb58a23e56c8fe85820c97337da887eddf4bb9bb
Parents: c448604
Author: Pankaj <pa...@huawei.com>
Authored: Tue Dec 18 00:49:22 2018 +0530
Committer: stack <st...@apache.org>
Committed: Tue Dec 18 20:51:01 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/HMaster.java     | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fb58a23e/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 a16e09d..0bcef59 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
@@ -900,11 +900,6 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   private void finishActiveMasterInitialization(MonitoredTask status) throws IOException,
           InterruptedException, KeeperException, ReplicationException {
-    Thread zombieDetector = new Thread(new InitializationMonitor(this),
-        "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
-    zombieDetector.setDaemon(true);
-    zombieDetector.start();
-
     /*
      * We are active master now... go initialize components we need to run.
      */
@@ -1001,6 +996,12 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Set ourselves as active Master now our claim has succeeded up in zk.
     this.activeMaster = true;
 
+    // Start the Zombie master detector after setting master as active, see HBASE-21535
+    Thread zombieDetector = new Thread(new InitializationMonitor(this),
+        "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
+    zombieDetector.setDaemon(true);
+    zombieDetector.start();
+
     // This is for backwards compatibility
     // See HBASE-11393
     status.setStatus("Update TableCFs node in ZNode");


[18/47] hbase git commit: HBASE-21565 Delete dead server from dead server list too early leads to concurrent Server Crash Procedures(SCP) for a same server

Posted by zh...@apache.org.
HBASE-21565 Delete dead server from dead server list too early leads to concurrent Server Crash Procedures(SCP) for a same server


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

Branch: refs/heads/HBASE-21512
Commit: c448604ceb987d113913f0583452b2abce04db0d
Parents: f782846
Author: Jingyun Tian <ti...@gmail.com>
Authored: Mon Dec 17 19:32:23 2018 +0800
Committer: Jingyun Tian <ti...@apache.org>
Committed: Tue Dec 18 16:57:11 2018 +0800

----------------------------------------------------------------------
 .../hbase/master/RegionServerTracker.java       |  3 +
 .../hadoop/hbase/master/ServerManager.java      | 25 ++++----
 .../master/assignment/AssignmentManager.java    | 28 ++++++---
 .../hbase/master/assignment/RegionStates.java   |  3 +-
 .../hbase/master/assignment/ServerState.java    |  2 +-
 .../master/assignment/ServerStateNode.java      |  2 +-
 .../master/procedure/ServerCrashProcedure.java  | 16 ++---
 .../hadoop/hbase/HBaseTestingUtility.java       |  7 ++-
 .../hadoop/hbase/master/TestRestartCluster.java | 65 ++++++++++++++++++++
 .../procedure/TestServerCrashProcedure.java     | 38 ++++++++++++
 10 files changed, 155 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
index f419732..9d33a21 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java
@@ -128,6 +128,9 @@ public class RegionServerTracker extends ZKListener {
     // '-SPLITTING'. Each splitting server should have a corresponding SCP. Log if not.
     splittingServersFromWALDir.stream().filter(s -> !deadServersFromPE.contains(s)).
       forEach(s -> LOG.error("{} has no matching ServerCrashProcedure", s));
+    //create ServerNode for all possible live servers from wal directory
+    liveServersFromWALDir.stream()
+        .forEach(sn -> server.getAssignmentManager().getRegionStates().getOrCreateServer(sn));
     watcher.registerListener(this);
     synchronized (this) {
       List<String> servers =

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index dc76d72..86d72d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -602,19 +602,22 @@ public class ServerManager {
       return false;
     }
     LOG.info("Processing expiration of " + serverName + " on " + this.master.getServerName());
-    master.getAssignmentManager().submitServerCrash(serverName, true);
-
-    // Tell our listeners that a server was removed
-    if (!this.listeners.isEmpty()) {
-      for (ServerListener listener : this.listeners) {
-        listener.serverRemoved(serverName);
+    long pid = master.getAssignmentManager().submitServerCrash(serverName, true);
+    if(pid <= 0) {
+      return false;
+    } else {
+      // Tell our listeners that a server was removed
+      if (!this.listeners.isEmpty()) {
+        for (ServerListener listener : this.listeners) {
+          listener.serverRemoved(serverName);
+        }
       }
+      // trigger a persist of flushedSeqId
+      if (flushedSeqIdFlusher != null) {
+        flushedSeqIdFlusher.triggerNow();
+      }
+      return true;
     }
-    // trigger a persist of flushedSeqId
-    if (flushedSeqIdFlusher != null) {
-      flushedSeqIdFlusher.triggerNow();
-    }
-    return true;
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index a564ea9..b7c2203 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -1343,24 +1343,36 @@ public class AssignmentManager {
   public long submitServerCrash(ServerName serverName, boolean shouldSplitWal) {
     boolean carryingMeta;
     long pid;
-    ServerStateNode serverNode = regionStates.getOrCreateServer(serverName);
+    ServerStateNode serverNode = regionStates.getServerNode(serverName);
+    if(serverNode == null){
+      LOG.info("Skip to add SCP for {} since this server should be OFFLINE already", serverName);
+      return -1;
+    }
     // we hold the write lock here for fencing on reportRegionStateTransition. Once we set the
     // server state to CRASHED, we will no longer accept the reportRegionStateTransition call from
     // this server. This is used to simplify the implementation for TRSP and SCP, where we can make
     // sure that, the region list fetched by SCP will not be changed any more.
     serverNode.writeLock().lock();
     try {
-      serverNode.setState(ServerState.CRASHED);
-      carryingMeta = isCarryingMeta(serverName);
       ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
-      pid = procExec.submitProcedure(new ServerCrashProcedure(procExec.getEnvironment(), serverName,
-        shouldSplitWal, carryingMeta));
+      carryingMeta = isCarryingMeta(serverName);
+      if (!serverNode.isInState(ServerState.ONLINE)) {
+        LOG.info(
+          "Skip to add SCP for {} with meta= {}, " +
+              "since there should be a SCP is processing or already done for this server node",
+          serverName, carryingMeta);
+        return -1;
+      } else {
+        serverNode.setState(ServerState.CRASHED);
+        pid = procExec.submitProcedure(new ServerCrashProcedure(procExec.getEnvironment(),
+            serverName, shouldSplitWal, carryingMeta));
+        LOG.info(
+          "Added {} to dead servers which carryingMeta={}, submitted ServerCrashProcedure pid={}",
+          serverName, carryingMeta, pid);
+      }
     } finally {
       serverNode.writeLock().unlock();
     }
-    LOG.info(
-      "Added {} to dead servers which carryingMeta={}, submitted ServerCrashProcedure pid={}",
-      serverName, carryingMeta, pid);
     return pid;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index 7b85409..1470a5a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -738,7 +738,8 @@ public class RegionStates {
     serverMap.remove(serverName);
   }
 
-  ServerStateNode getServerNode(final ServerName serverName) {
+  @VisibleForTesting
+  public ServerStateNode getServerNode(final ServerName serverName) {
     return serverMap.get(serverName);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
index 3efe6e2..c86a60e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
@@ -23,7 +23,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Server State.
  */
 @InterfaceAudience.Private
-enum ServerState {
+public enum ServerState {
   /**
    * Initial state. Available.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
index 6f763aa..11883db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
@@ -32,7 +32,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * State of Server; list of hosted regions, etc.
  */
 @InterfaceAudience.Private
-class ServerStateNode implements Comparable<ServerStateNode> {
+public class ServerStateNode implements Comparable<ServerStateNode> {
 
   private final Set<RegionStateNode> regions;
   private final ServerName serverName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index b93f8fa..05bcd28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -333,17 +333,6 @@ public class ServerCrashProcedure
     return ServerOperationType.CRASH_HANDLER;
   }
 
-  /**
-   * For this procedure, yield at end of each successful flow step so that all crashed servers
-   * can make progress rather than do the default which has each procedure running to completion
-   * before we move to the next. For crashed servers, especially if running with distributed log
-   * replay, we will want all servers to come along; we do not want the scenario where a server is
-   * stuck waiting for regions to online so it can replay edits.
-   */
-  @Override
-  protected boolean isYieldBeforeExecuteFromState(MasterProcedureEnv env, ServerCrashState state) {
-    return true;
-  }
 
   @Override
   protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
@@ -390,4 +379,9 @@ public class ServerCrashProcedure
   protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) {
     return env.getMasterServices().getMasterMetrics().getServerCrashProcMetrics();
   }
+
+  @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/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 0cd5a22..7bfbfe1 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
@@ -1187,6 +1187,11 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * @param servers number of region servers
    */
   public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
+    this.restartHBaseCluster(servers, null);
+  }
+
+  public void restartHBaseCluster(int servers, List<Integer> ports)
+      throws IOException, InterruptedException {
     if (hbaseAdmin != null) {
       hbaseAdmin.close();
       hbaseAdmin = null;
@@ -1195,7 +1200,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       this.connection.close();
       this.connection = null;
     }
-    this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
+    this.hbaseCluster = new MiniHBaseCluster(this.conf, 1, servers, ports, null, null);
     // Don't leave here till we've done a successful scan of the hbase:meta
     Connection conn = ConnectionFactory.createConnection(this.conf);
     Table t = conn.getTable(TableName.META_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
index 4ba1876..e55e375 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
@@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
+
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -33,12 +35,18 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.assignment.ServerState;
+import org.apache.hadoop.hbase.master.assignment.ServerStateNode;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -67,6 +75,63 @@ public class TestRestartCluster {
   }
 
   @Test
+  public void testClusterRestartFailOver() throws Exception {
+    UTIL.startMiniCluster(3);
+    UTIL.waitFor(60000, () -> UTIL.getMiniHBaseCluster().getMaster().isInitialized());
+    //wait for all SCPs finished
+    UTIL.waitFor(20000, () -> UTIL.getHBaseCluster().getMaster().getProcedures().stream()
+        .noneMatch(p -> p instanceof ServerCrashProcedure));
+    TableName tableName = TABLES[0];
+    ServerName testServer = UTIL.getHBaseCluster().getRegionServer(0).getServerName();
+    ServerStateNode serverNode = UTIL.getHBaseCluster().getMaster().getAssignmentManager()
+        .getRegionStates().getServerNode(testServer);
+    Assert.assertNotNull(serverNode);
+    Assert.assertTrue("serverNode should be ONLINE when cluster runs normally",
+      serverNode.isInState(ServerState.ONLINE));
+    UTIL.createMultiRegionTable(tableName, FAMILY);
+    UTIL.waitTableEnabled(tableName);
+    Table table = UTIL.getConnection().getTable(tableName);
+    for (int i = 0; i < 100; i++) {
+      UTIL.loadTable(table, FAMILY);
+    }
+    List<Integer> ports =
+        UTIL.getHBaseCluster().getMaster().getServerManager().getOnlineServersList().stream()
+            .map(serverName -> serverName.getPort()).collect(Collectors.toList());
+    LOG.info("Shutting down cluster");
+    UTIL.getHBaseCluster().killAll();
+    UTIL.getHBaseCluster().waitUntilShutDown();
+    LOG.info("Starting cluster the second time");
+    UTIL.restartHBaseCluster(3, ports);
+    UTIL.waitFor(10000, () -> UTIL.getHBaseCluster().getMaster().isInitialized());
+    serverNode = UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
+        .getServerNode(testServer);
+    Assert.assertNotNull("serverNode should not be null when restart whole cluster", serverNode);
+    Assert.assertFalse(serverNode.isInState(ServerState.ONLINE));
+    LOG.info("start to find the procedure of SCP for the severName we choose");
+    UTIL.waitFor(20000,
+      () -> UTIL.getHBaseCluster().getMaster().getProcedures().stream()
+          .anyMatch(procedure -> (procedure instanceof ServerCrashProcedure)
+              && ((ServerCrashProcedure) procedure).getServerName().equals(testServer)));
+    Assert.assertFalse("serverNode should not be ONLINE during SCP processing",
+      serverNode.isInState(ServerState.ONLINE));
+    LOG.info("start to submit the SCP for the same serverName {} which should fail", testServer);
+    Assert.assertFalse(
+      UTIL.getHBaseCluster().getMaster().getServerManager().expireServer(testServer));
+    Procedure procedure = UTIL.getHBaseCluster().getMaster().getProcedures().stream()
+        .filter(p -> (p instanceof ServerCrashProcedure)
+            && ((ServerCrashProcedure) p).getServerName().equals(testServer))
+        .findAny().get();
+    UTIL.waitFor(20000, () -> procedure.isFinished());
+    LOG.info("even when the SCP is finished, the duplicate SCP should not be scheduled for {}",
+      testServer);
+    Assert.assertFalse(
+      UTIL.getHBaseCluster().getMaster().getServerManager().expireServer(testServer));
+    serverNode = UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
+        .getServerNode(testServer);
+    Assert.assertNull("serverNode should be deleted after SCP finished", serverNode);
+  }
+
+  @Test
   public void testClusterRestart() throws Exception {
     UTIL.startMiniCluster(3);
     while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c448604c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
index 0e4a84b..af2076e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
+import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -170,6 +171,43 @@ public class TestServerCrashProcedure {
     }
   }
 
+  @Test
+  public void testConcurrentSCPForSameServer() throws Exception {
+    final TableName tableName = TableName.valueOf("testConcurrentSCPForSameServer");
+    try (Table t = createTable(tableName)) {
+      // Load the table with a bit of data so some logs to split and some edits in each region.
+      this.util.loadTable(t, HBaseTestingUtility.COLUMNS[0]);
+      final int count = util.countRows(t);
+      assertTrue("expected some rows", count > 0);
+      // find the first server that match the request and executes the test
+      ServerName rsToKill = null;
+      for (RegionInfo hri : util.getAdmin().getRegions(tableName)) {
+        final ServerName serverName = AssignmentTestingUtil.getServerHoldingRegion(util, hri);
+        if (AssignmentTestingUtil.isServerHoldingMeta(util, serverName) == true) {
+          rsToKill = serverName;
+          break;
+        }
+      }
+      HMaster master = util.getHBaseCluster().getMaster();
+      final ProcedureExecutor<MasterProcedureEnv> pExecutor = master.getMasterProcedureExecutor();
+      ServerCrashProcedure procB =
+          new ServerCrashProcedure(pExecutor.getEnvironment(), rsToKill, false, false);
+      AssignmentTestingUtil.killRs(util, rsToKill);
+      long procId = getSCPProcId(pExecutor);
+      Procedure procA = pExecutor.getProcedure(procId);
+      LOG.info("submit SCP procedureA");
+      util.waitFor(5000, () -> procA.hasLock());
+      LOG.info("procedureA acquired the lock");
+      assertEquals(Procedure.LockState.LOCK_EVENT_WAIT,
+          procB.acquireLock(pExecutor.getEnvironment()));
+      LOG.info("procedureB should not be able to get the lock");
+      util.waitFor(60000,
+        () -> procB.acquireLock(pExecutor.getEnvironment()) == Procedure.LockState.LOCK_ACQUIRED);
+      LOG.info("when procedure B get the lock, procedure A should be finished");
+      assertTrue(procA.isFinished());
+    }
+  }
+
   protected void assertReplicaDistributed(final Table t) {
     return;
   }


[34/47] hbase git commit: HBASE-21643 Introduce two new region coprocessor method and deprecated postMutationBeforeWAL

Posted by zh...@apache.org.
HBASE-21643 Introduce two new region coprocessor method and deprecated postMutationBeforeWAL


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

Branch: refs/heads/HBASE-21512
Commit: f5ea00f72442e5c80f2a5fc6e99506127fa8d16b
Parents: c2d5991
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Dec 26 17:42:02 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu Dec 27 18:27:06 2018 +0800

----------------------------------------------------------------------
 .../hbase/coprocessor/RegionObserver.java       | 47 ++++++++++++++++++++
 .../hadoop/hbase/regionserver/HRegion.java      | 26 ++++++-----
 .../regionserver/RegionCoprocessorHost.java     | 29 +++++++++---
 .../hbase/security/access/AccessController.java | 30 ++++++++++---
 .../visibility/VisibilityController.java        | 30 +++++++++++--
 5 files changed, 134 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f5ea00f7/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index c14cbd1..95b2150 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.coprocessor;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -1029,13 +1030,59 @@ public interface RegionObserver {
    * @param oldCell old cell containing previous value
    * @param newCell the new cell containing the computed value
    * @return the new cell, possibly changed
+   * @deprecated Use {@link #postIncrementBeforeWAL} or {@link #postAppendBeforeWAL} instead.
    */
+  @Deprecated
   default Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
       MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
     return newCell;
   }
 
   /**
+   * Called after a list of new cells has been created during an increment operation, but before
+   * they are committed to the WAL or memstore.
+   *
+   * @param ctx       the environment provided by the region server
+   * @param mutation  the current mutation
+   * @param cellPairs a list of cell pair. The first cell is old cell which may be null.
+   *                  And the second cell is the new cell.
+   * @return a list of cell pair, possibly changed.
+   */
+  default List<Pair<Cell, Cell>> postIncrementBeforeWAL(
+      ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
+      List<Pair<Cell, Cell>> cellPairs) throws IOException {
+    List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size());
+    for (Pair<Cell, Cell> pair : cellPairs) {
+      resultPairs.add(new Pair<>(pair.getFirst(),
+          postMutationBeforeWAL(ctx, MutationType.INCREMENT, mutation, pair.getFirst(),
+              pair.getSecond())));
+    }
+    return resultPairs;
+  }
+
+  /**
+   * Called after a list of new cells has been created during an append operation, but before
+   * they are committed to the WAL or memstore.
+   *
+   * @param ctx       the environment provided by the region server
+   * @param mutation  the current mutation
+   * @param cellPairs a list of cell pair. The first cell is old cell which may be null.
+   *                  And the second cell is the new cell.
+   * @return a list of cell pair, possibly changed.
+   */
+  default List<Pair<Cell, Cell>> postAppendBeforeWAL(
+      ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
+      List<Pair<Cell, Cell>> cellPairs) throws IOException {
+    List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size());
+    for (Pair<Cell, Cell> pair : cellPairs) {
+      resultPairs.add(new Pair<>(pair.getFirst(),
+          postMutationBeforeWAL(ctx, MutationType.INCREMENT, mutation, pair.getFirst(),
+              pair.getSecond())));
+    }
+    return resultPairs;
+  }
+
+  /**
    * Called after the ScanQueryMatcher creates ScanDeleteTracker. Implementing
    * this hook would help in creating customised DeleteTracker and returning
    * the newly created DeleteTracker

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5ea00f7/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 9bf9309..ec222c7 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
@@ -70,6 +70,8 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
+import java.util.stream.Collectors;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -120,7 +122,6 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver.MutationType;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
@@ -8014,7 +8015,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       Durability effectiveDurability, long now, List<Cell> deltas, List<Cell> results)
       throws IOException {
     byte[] columnFamily = store.getColumnFamilyDescriptor().getName();
-    List<Cell> toApply = new ArrayList<>(deltas.size());
+    List<Pair<Cell, Cell>> cellPairs = new ArrayList<>(deltas.size());
     // Get previous values for all columns in this family.
     TimeRange tr = null;
     switch (op) {
@@ -8041,18 +8042,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           currentValuesIndex++;
         }
       }
+
       // Switch on whether this an increment or an append building the new Cell to apply.
       Cell newCell = null;
-      MutationType mutationType = null;
       switch (op) {
         case INCREMENT:
-          mutationType = MutationType.INCREMENT;
           long deltaAmount = getLongValue(delta);
           final long newValue = currentValue == null ? deltaAmount : getLongValue(currentValue) + deltaAmount;
           newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, (oldCell) -> Bytes.toBytes(newValue));
           break;
         case APPEND:
-          mutationType = MutationType.APPEND;
           newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, (oldCell) ->
             ByteBuffer.wrap(new byte[delta.getValueLength() + oldCell.getValueLength()])
                     .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength())
@@ -8063,18 +8062,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         default: throw new UnsupportedOperationException(op.toString());
       }
 
-      // Give coprocessors a chance to update the new cell
-      if (coprocessorHost != null) {
-        newCell =
-            coprocessorHost.postMutationBeforeWAL(mutationType, mutation, currentValue, newCell);
-      }
-      toApply.add(newCell);
+      cellPairs.add(new Pair<>(currentValue, newCell));
       // Add to results to get returned to the Client. If null, cilent does not want results.
       if (results != null) {
         results.add(newCell);
       }
     }
-    return toApply;
+
+    // Give coprocessors a chance to update the new cells before apply to WAL or memstore
+    if (coprocessorHost != null) {
+      // Here the operation must be increment or append.
+      cellPairs = op == Operation.INCREMENT ?
+          coprocessorHost.postIncrementBeforeWAL(mutation, cellPairs) :
+          coprocessorHost.postAppendBeforeWAL(mutation, cellPairs);
+    }
+    return cellPairs.stream().map(Pair::getSecond).collect(Collectors.toList());
   }
 
   private static Cell reckonDelta(final Cell delta, final Cell currentCell,

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5ea00f7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
index dea13ca..16fd332 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
@@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver.MutationType;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.Reference;
@@ -1691,16 +1690,32 @@ public class RegionCoprocessorHost
         });
   }
 
-  public Cell postMutationBeforeWAL(final MutationType opType, final Mutation mutation,
-      final Cell oldCell, Cell newCell) throws IOException {
+  public List<Pair<Cell, Cell>> postIncrementBeforeWAL(final Mutation mutation,
+      final List<Pair<Cell, Cell>> cellPairs) throws IOException {
     if (this.coprocEnvironments.isEmpty()) {
-      return newCell;
+      return cellPairs;
     }
     return execOperationWithResult(
-        new ObserverOperationWithResult<RegionObserver, Cell>(regionObserverGetter, newCell) {
+        new ObserverOperationWithResult<RegionObserver, List<Pair<Cell, Cell>>>(
+            regionObserverGetter, cellPairs) {
           @Override
-          public Cell call(RegionObserver observer) throws IOException {
-            return observer.postMutationBeforeWAL(this, opType, mutation, oldCell, getResult());
+          public List<Pair<Cell, Cell>> call(RegionObserver observer) throws IOException {
+            return observer.postIncrementBeforeWAL(this, mutation, getResult());
+          }
+        });
+  }
+
+  public List<Pair<Cell, Cell>> postAppendBeforeWAL(final Mutation mutation,
+      final List<Pair<Cell, Cell>> cellPairs) throws IOException {
+    if (this.coprocEnvironments.isEmpty()) {
+      return cellPairs;
+    }
+    return execOperationWithResult(
+        new ObserverOperationWithResult<RegionObserver, List<Pair<Cell, Cell>>>(
+            regionObserverGetter, cellPairs) {
+          @Override
+          public List<Pair<Cell, Cell>> call(RegionObserver observer) throws IOException {
+            return observer.postAppendBeforeWAL(this, mutation, getResult());
           }
         });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5ea00f7/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 82ec12d..6e2c9ce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -36,6 +36,7 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ArrayBackedTag;
@@ -1849,14 +1850,34 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   @Override
-  public Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
-      MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
+  public List<Pair<Cell, Cell>> postIncrementBeforeWAL(
+      ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
+      List<Pair<Cell, Cell>> cellPairs) throws IOException {
     // If the HFile version is insufficient to persist tags, we won't have any
     // work to do here
     if (!cellFeaturesEnabled) {
-      return newCell;
+      return cellPairs;
     }
+    return cellPairs.stream().map(pair -> new Pair<>(pair.getFirst(),
+        createNewCellWithTags(mutation, pair.getFirst(), pair.getSecond())))
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public List<Pair<Cell, Cell>> postAppendBeforeWAL(
+      ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
+      List<Pair<Cell, Cell>> cellPairs) throws IOException {
+    // If the HFile version is insufficient to persist tags, we won't have any
+    // work to do here
+    if (!cellFeaturesEnabled) {
+      return cellPairs;
+    }
+    return cellPairs.stream().map(pair -> new Pair<>(pair.getFirst(),
+        createNewCellWithTags(mutation, pair.getFirst(), pair.getSecond())))
+        .collect(Collectors.toList());
+  }
 
+  private Cell createNewCellWithTags(Mutation mutation, Cell oldCell, Cell newCell) {
     // Collect any ACLs from the old cell
     List<Tag> tags = Lists.newArrayList();
     List<Tag> aclTags = Lists.newArrayList();
@@ -1901,8 +1922,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       return newCell;
     }
 
-    Cell rewriteCell = PrivateCellUtil.createCell(newCell, tags);
-    return rewriteCell;
+    return PrivateCellUtil.createCell(newCell, tags);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f5ea00f7/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index c4f3b95..2a18551 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -127,6 +127,7 @@ import org.slf4j.LoggerFactory;
 public class VisibilityController implements MasterCoprocessor, RegionCoprocessor,
     VisibilityLabelsService.Interface, MasterObserver, RegionObserver {
 
+
   private static final Logger LOG = LoggerFactory.getLogger(VisibilityController.class);
   private static final Logger AUDITLOG = LoggerFactory.getLogger("SecurityLogger."
       + VisibilityController.class.getName());
@@ -688,8 +689,30 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
   }
 
   @Override
-  public Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
-      MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
+  public List<Pair<Cell, Cell>> postIncrementBeforeWAL(
+      ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
+      List<Pair<Cell, Cell>> cellPairs) throws IOException {
+    List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size());
+    for (Pair<Cell, Cell> pair : cellPairs) {
+      resultPairs
+          .add(new Pair<>(pair.getFirst(), createNewCellWithTags(mutation, pair.getSecond())));
+    }
+    return resultPairs;
+  }
+
+  @Override
+  public List<Pair<Cell, Cell>> postAppendBeforeWAL(
+      ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
+      List<Pair<Cell, Cell>> cellPairs) throws IOException {
+    List<Pair<Cell, Cell>> resultPairs = new ArrayList<>(cellPairs.size());
+    for (Pair<Cell, Cell> pair : cellPairs) {
+      resultPairs
+          .add(new Pair<>(pair.getFirst(), createNewCellWithTags(mutation, pair.getSecond())));
+    }
+    return resultPairs;
+  }
+
+  private Cell createNewCellWithTags(Mutation mutation, Cell newCell) throws IOException {
     List<Tag> tags = Lists.newArrayList();
     CellVisibility cellVisibility = null;
     try {
@@ -715,8 +738,7 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
       }
     }
 
-    Cell rewriteCell = PrivateCellUtil.createCell(newCell, tags);
-    return rewriteCell;
+    return PrivateCellUtil.createCell(newCell, tags);
   }
 
   @Override