You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2017/09/24 00:34:01 UTC

[01/47] hbase git commit: HBASE-18772 [JDK8] Replace AtomicLong with LongAdder; ADDENDUM -- disable broke test [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-18467 244470ca3 -> 338446515 (forced update)


HBASE-18772 [JDK8] Replace AtomicLong with LongAdder; ADDENDUM -- disable broke test


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

Branch: refs/heads/HBASE-18467
Commit: d2792a07fe4fe9b10329e52176a0ad4bc0bd0f79
Parents: 780be08
Author: Michael Stack <st...@apache.org>
Authored: Wed Sep 13 23:23:31 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Sep 13 23:24:53 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java   | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d2792a07/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index b4f93c5..90d5772 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -430,6 +431,7 @@ public class TestSimpleRpcScheduler {
   // fastpath thread to: new FastPathBalancedQueueRpcExecutor("CodelFPBQ.default", handlerCount, numCallQueues...
   // Codel is hard to test. This test is going to be flakey given it all timer-based. Disabling for now till chat
   // with authors.
+  @Ignore
   @Test
   public void testCoDelScheduling() throws Exception {
     CoDelEnvironmentEdge envEdge = new CoDelEnvironmentEdge();


[05/47] hbase git commit: HBASE-18453 CompactionRequest should not be exposed to user directly

Posted by bu...@apache.org.
HBASE-18453 CompactionRequest should not be exposed to user directly


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

Branch: refs/heads/HBASE-18467
Commit: 61d10feffaa7b96ee46e2a6f1e542d80c1d76f42
Parents: 38e983e
Author: zhangduo <zh...@apache.org>
Authored: Mon Sep 11 08:50:37 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Sep 14 20:37:33 2017 +0800

----------------------------------------------------------------------
 .../example/ZooKeeperScanPolicyObserver.java    |   3 +-
 .../example/TestRefreshHFilesEndpoint.java      |  11 +-
 .../hbase/regionserver/CompactionTool.java      |  11 +-
 .../hbase/coprocessor/RegionObserver.java       |  36 +-
 .../hadoop/hbase/regionserver/CompactSplit.java | 233 ++++----
 .../hbase/regionserver/CompactionRequestor.java | 100 ----
 .../regionserver/FlushAllLargeStoresPolicy.java |  18 +-
 .../regionserver/FlushAllStoresPolicy.java      |   2 +-
 .../regionserver/FlushLargeStoresPolicy.java    |   2 +-
 .../FlushNonSloppyStoresFirstPolicy.java        |  29 +-
 .../hadoop/hbase/regionserver/FlushPolicy.java  |   2 +-
 .../hadoop/hbase/regionserver/HRegion.java      | 590 +++++++++----------
 .../hbase/regionserver/HRegionServer.java       |  53 +-
 .../hadoop/hbase/regionserver/HStore.java       |  60 +-
 .../hbase/regionserver/MemStoreFlusher.java     |  12 +-
 .../MetricsRegionServerWrapperImpl.java         |   2 +-
 .../regionserver/MetricsRegionWrapperImpl.java  |   2 +-
 .../hbase/regionserver/RSRpcServices.java       |  20 +-
 .../hadoop/hbase/regionserver/Region.java       |  34 +-
 .../regionserver/RegionCoprocessorHost.java     |  60 +-
 .../regionserver/RegionServerServices.java      |   5 -
 .../hbase/regionserver/RegionSplitPolicy.java   |   2 +-
 .../apache/hadoop/hbase/regionserver/Store.java |  20 +-
 .../compactions/CompactionLifeCycleTracker.java |  52 ++
 .../compactions/CompactionRequest.java          |  71 +--
 .../regionserver/compactions/Compactor.java     |  15 +-
 .../hbase/security/access/AccessController.java |   4 +-
 .../hbase-webapps/regionserver/region.jsp       |   2 +-
 .../hadoop/hbase/MockRegionServerServices.java  |  14 +-
 .../org/apache/hadoop/hbase/TestIOFencing.java  |   4 +-
 ...estAvoidCellReferencesIntoShippedBlocks.java |   4 +-
 .../client/TestBlockEvictionFromClient.java     |   2 +-
 .../hbase/coprocessor/SimpleRegionObserver.java |  13 +-
 .../coprocessor/TestCoprocessorInterface.java   |   5 +-
 .../TestRegionObserverInterface.java            |   6 +-
 .../TestRegionObserverScannerOpenHook.java      |   8 +-
 .../hadoop/hbase/master/MockRegionServer.java   |  52 +-
 .../hbase/mob/compactions/TestMobCompactor.java |  12 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |   3 +-
 .../quotas/TestFileSystemUtilizationChore.java  |   6 +-
 .../regionserver/NoOpScanPolicyObserver.java    |   3 +-
 .../regionserver/StatefulStoreMockMaker.java    |  43 +-
 .../hbase/regionserver/TestCompaction.java      |  70 +--
 .../hbase/regionserver/TestHMobStore.java       |   7 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   4 +-
 .../hbase/regionserver/TestMajorCompaction.java |   7 +-
 .../TestSplitTransactionOnCluster.java          |  15 +-
 .../regionserver/TestSplitWalDataLoss.java      |   2 +-
 .../hadoop/hbase/regionserver/TestStore.java    |  17 +-
 .../regionserver/wal/AbstractTestWALReplay.java |   6 +-
 .../hbase/util/TestCoprocessorScanPolicy.java   |   3 +-
 51 files changed, 800 insertions(+), 957 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
index 344d188..6b31664 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -203,7 +204,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
   @Override
   public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
       Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
+      InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
     ScanInfo scanInfo = getScanInfo(store, c.getEnvironment());
     if (scanInfo == null) {
       // take default action

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
index a037f85..257b075 100644
--- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
+++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesEndpoint.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 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.RetriesExhaustedException;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -144,15 +145,17 @@ public class TestRefreshHFilesEndpoint {
     }
 
     @Override
-    public List<Store> getStores() {
-      List<Store> list = new ArrayList<Store>(stores.size());
+    public List<HStore> getStores() {
+      List<HStore> list = new ArrayList<>(stores.size());
       /**
        * This is used to trigger the custom definition (faulty)
        * of refresh HFiles API.
        */
       try {
-        if (this.store == null)
-          store = new HStoreWithFaultyRefreshHFilesAPI(this, new HColumnDescriptor(FAMILY), this.conf);
+        if (this.store == null) {
+          store = new HStoreWithFaultyRefreshHFilesAPI(this,
+              ColumnFamilyDescriptorBuilder.of(FAMILY), this.conf);
+        }
         list.add(store);
       } catch (IOException ioe) {
         LOG.info("Couldn't instantiate custom store implementation", ioe);

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index de59c20..bb01459 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Optional;
 import java.util.Set;
 
 import org.apache.commons.logging.Log;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.mapreduce.JobUtil;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -158,10 +160,13 @@ public class CompactionTool extends Configured implements Tool {
         store.triggerMajorCompaction();
       }
       do {
-        CompactionContext compaction = store.requestCompaction(Store.PRIORITY_USER, null);
-        if (compaction == null) break;
+        Optional<CompactionContext> compaction =
+            store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null);
+        if (!compaction.isPresent()) {
+          break;
+        }
         List<StoreFile> storeFiles =
-            store.compact(compaction, NoLimitThroughputController.INSTANCE);
+            store.compact(compaction.get(), NoLimitThroughputController.INSTANCE);
         if (storeFiles != null && !storeFiles.isEmpty()) {
           if (keepCompactedFiles && deleteCompacted) {
             for (StoreFile storeFile: storeFiles) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 b036608..ae57747 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
@@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.util.Pair;
@@ -186,10 +186,10 @@ public interface RegionObserver extends Coprocessor {
    * @param c the environment provided by the region server
    * @param store the store where compaction is being requested
    * @param candidates the store files currently available for compaction
-   * @param request custom compaction request
+   * @param tracker tracker used to track the life cycle of a compaction
    */
   default void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      List<StoreFile> candidates, CompactionRequest request) throws IOException {}
+      List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {}
 
   /**
    * Called after the {@link StoreFile}s to compact have been selected from the available
@@ -197,10 +197,10 @@ public interface RegionObserver extends Coprocessor {
    * @param c the environment provided by the region server
    * @param store the store being compacted
    * @param selected the store files selected to compact
-   * @param request custom compaction request
+   * @param tracker tracker used to track the life cycle of a compaction
    */
   default void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      ImmutableList<StoreFile> selected, CompactionRequest request) {}
+      ImmutableList<StoreFile> selected, CompactionLifeCycleTracker tracker) {}
 
   /**
    * Called prior to writing the {@link StoreFile}s selected for compaction into a new
@@ -220,13 +220,13 @@ public interface RegionObserver extends Coprocessor {
    * @param store the store being compacted
    * @param scanner the scanner over existing data used in the store file rewriting
    * @param scanType type of Scan
-   * @param request the requested compaction
+   * @param tracker tracker used to track the life cycle of a compaction
    * @return the scanner to use during compaction. Should not be {@code null} unless the
    *         implementation is writing new store files on its own.
    */
-  default InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, InternalScanner scanner, ScanType scanType,
-      CompactionRequest request) throws IOException {
+  default InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+      InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker)
+      throws IOException {
     return scanner;
   }
 
@@ -245,14 +245,14 @@ public interface RegionObserver extends Coprocessor {
    * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
    *          files
    * @param s the base scanner, if not {@code null}, from previous RegionObserver in the chain
-   * @param request compaction request
+   * @param tracker used to track the life cycle of a compaction
    * @param readPoint the readpoint to create scanner
    * @return the scanner to use during compaction. {@code null} if the default implementation is to
    *          be used.
    */
   default InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
       Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
+      InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
     return s;
   }
 
@@ -261,10 +261,10 @@ public interface RegionObserver extends Coprocessor {
    * @param c the environment provided by the region server
    * @param store the store being compacted
    * @param resultFile the new store file written out during compaction
-   * @param request the requested compaction
+   * @param tracker used to track the life cycle of a compaction
    */
   default void postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      StoreFile resultFile, CompactionRequest request) throws IOException {}
+      StoreFile resultFile, CompactionLifeCycleTracker tracker) throws IOException {}
 
   /**
    * Called before the region is reported as closed to the master.
@@ -798,12 +798,12 @@ public interface RegionObserver extends Coprocessor {
    * Called before a store opens a new scanner.
    * This hook is called when a "user" scanner is opened.
    * <p>
-   * See {@link #preFlushScannerOpen(ObserverContext, Store, List, InternalScanner, long)} and {@link #preCompactScannerOpen(ObserverContext,
-   *  Store, List, ScanType, long, InternalScanner, CompactionRequest, long)}
-   * to override scanners created for flushes or compactions, resp.
+   * See {@link #preFlushScannerOpen(ObserverContext, Store, List, InternalScanner, long)}
+   * and {@link #preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
+   * InternalScanner, CompactionLifeCycleTracker, long)} to override scanners created for flushes
+   * or compactions, resp.
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors.
+   * Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors.
    * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
    * effect in this hook.
    * <p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index 621bead..cdeeff7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -21,10 +21,9 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.Iterator;
-import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Executors;
 import java.util.concurrent.RejectedExecutionException;
@@ -41,24 +40,23 @@ import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.StealJobQueue;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-
 /**
  * Compact region on request and then run split if appropriate
  */
 @InterfaceAudience.Private
-public class CompactSplit implements CompactionRequestor, PropagatingConfigurationObserver {
+public class CompactSplit implements PropagatingConfigurationObserver {
   private static final Log LOG = LogFactory.getLog(CompactSplit.class);
 
   // Configuration key for the large compaction threads.
@@ -233,126 +231,89 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
     }
   }
 
-  @Override
-  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why)
-      throws IOException {
-    return requestCompaction(r, why, null);
+  public synchronized void requestCompaction(HRegion region, String why, int priority,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
+    requestCompactionInternal(region, why, priority, true, tracker, user);
   }
 
-  @Override
-  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
-      List<Pair<CompactionRequest, Store>> requests) throws IOException {
-    return requestCompaction(r, why, Store.NO_PRIORITY, requests, null);
-  }
-
-  @Override
-  public synchronized CompactionRequest requestCompaction(final Region r, final Store s,
-      final String why, CompactionRequest request) throws IOException {
-    return requestCompaction(r, s, why, Store.NO_PRIORITY, request, null);
+  public synchronized void requestCompaction(HRegion region, HStore store, String why, int priority,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
+    requestCompactionInternal(region, store, why, priority, true, tracker, user);
   }
 
-  @Override
-  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
-      int p, List<Pair<CompactionRequest, Store>> requests, User user) throws IOException {
-    return requestCompactionInternal(r, why, p, requests, true, user);
-  }
-
-  private List<CompactionRequest> requestCompactionInternal(final Region r, final String why,
-      int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow, User user)
-          throws IOException {
-    // not a special compaction request, so make our own list
-    List<CompactionRequest> ret = null;
-    if (requests == null) {
-      ret = selectNow ? new ArrayList<CompactionRequest>(r.getStores().size()) : null;
-      for (Store s : r.getStores()) {
-        CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow, user);
-        if (selectNow) ret.add(cr);
-      }
-    } else {
-      Preconditions.checkArgument(selectNow); // only system requests have selectNow == false
-      ret = new ArrayList<CompactionRequest>(requests.size());
-      for (Pair<CompactionRequest, Store> pair : requests) {
-        ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst(), user));
-      }
+  private void requestCompactionInternal(HRegion region, String why, int priority,
+      boolean selectNow, CompactionLifeCycleTracker tracker, User user) throws IOException {
+    // request compaction on all stores
+    for (HStore store : region.stores.values()) {
+      requestCompactionInternal(region, store, why, priority, selectNow, tracker, user);
     }
-    return ret;
-  }
-
-  public CompactionRequest requestCompaction(final Region r, final Store s,
-      final String why, int priority, CompactionRequest request, User user) throws IOException {
-    return requestCompactionInternal(r, s, why, priority, request, true, user);
-  }
-
-  public synchronized void requestSystemCompaction(
-      final Region r, final String why) throws IOException {
-    requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false, null);
-  }
-
-  public void requestSystemCompaction(
-      final Region r, final Store s, final String why) throws IOException {
-    requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false, null);
   }
 
-  /**
-   * @param r region store belongs to
-   * @param s Store to request compaction on
-   * @param why Why compaction requested -- used in debug messages
-   * @param priority override the default priority (NO_PRIORITY == decide)
-   * @param request custom compaction request. Can be <tt>null</tt> in which case a simple
-   *          compaction will be used.
-   */
-  private synchronized CompactionRequest requestCompactionInternal(final Region r, final Store s,
-      final String why, int priority, CompactionRequest request, boolean selectNow, User user)
-          throws IOException {
-    if (this.server.isStopped()
-        || (r.getTableDescriptor() != null && !r.getTableDescriptor().isCompactionEnabled())) {
-      return null;
+  private void requestCompactionInternal(HRegion region, HStore store, String why, int priority,
+      boolean selectNow, CompactionLifeCycleTracker tracker, User user) throws IOException {
+    if (this.server.isStopped() || (region.getTableDescriptor() != null &&
+        !region.getTableDescriptor().isCompactionEnabled())) {
+      return;
     }
-
-    CompactionContext compaction = null;
+    Optional<CompactionContext> compaction;
     if (selectNow) {
-      compaction = selectCompaction(r, s, priority, request, user);
-      if (compaction == null) return null; // message logged inside
+      compaction = selectCompaction(region, store, priority, tracker, user);
+      if (!compaction.isPresent()) {
+        // message logged inside
+        return;
+      }
+    } else {
+      compaction = Optional.empty();
     }
 
-    final RegionServerSpaceQuotaManager spaceQuotaManager =
-      this.server.getRegionServerSpaceQuotaManager();
-    if (spaceQuotaManager != null && spaceQuotaManager.areCompactionsDisabled(
-        r.getTableDescriptor().getTableName())) {
+    RegionServerSpaceQuotaManager spaceQuotaManager =
+        this.server.getRegionServerSpaceQuotaManager();
+    if (spaceQuotaManager != null &&
+        spaceQuotaManager.areCompactionsDisabled(region.getTableDescriptor().getTableName())) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "
-            + " policy disallows compactions.");
+        LOG.debug("Ignoring compaction request for " + region +
+            " as an active space quota violation " + " policy disallows compactions.");
       }
-      return null;
+      return;
     }
 
-    // We assume that most compactions are small. So, put system compactions into small
-    // pool; we will do selection there, and move to large pool if necessary.
-    ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))
-      ? longCompactions : shortCompactions;
-    pool.execute(new CompactionRunner(s, r, compaction, pool, user));
-    ((HRegion)r).incrementCompactionsQueuedCount();
+    ThreadPoolExecutor pool;
+    if (selectNow) {
+      // compaction.get is safe as we will just return if selectNow is true but no compaction is
+      // selected
+      pool = store.throttleCompaction(compaction.get().getRequest().getSize()) ? longCompactions
+          : shortCompactions;
+    } else {
+      // We assume that most compactions are small. So, put system compactions into small
+      // pool; we will do selection there, and move to large pool if necessary.
+      pool = shortCompactions;
+    }
+    pool.execute(new CompactionRunner(store, region, compaction, pool, user));
+    region.incrementCompactionsQueuedCount();
     if (LOG.isDebugEnabled()) {
       String type = (pool == shortCompactions) ? "Small " : "Large ";
       LOG.debug(type + "Compaction requested: " + (selectNow ? compaction.toString() : "system")
           + (why != null && !why.isEmpty() ? "; Because: " + why : "") + "; " + this);
     }
-    return selectNow ? compaction.getRequest() : null;
   }
 
-  private CompactionContext selectCompaction(final Region r, final Store s,
-      int priority, CompactionRequest request, User user) throws IOException {
-    CompactionContext compaction = s.requestCompaction(priority, request, user);
-    if (compaction == null) {
-      if(LOG.isDebugEnabled() && r.getRegionInfo() != null) {
-        LOG.debug("Not compacting " + r.getRegionInfo().getRegionNameAsString() +
-            " because compaction request was cancelled");
-      }
-      return null;
-    }
-    assert compaction.hasSelection();
-    if (priority != Store.NO_PRIORITY) {
-      compaction.getRequest().setPriority(priority);
+  public synchronized void requestSystemCompaction(HRegion region, String why) throws IOException {
+    requestCompactionInternal(region, why, Store.NO_PRIORITY, false,
+      CompactionLifeCycleTracker.DUMMY, null);
+  }
+
+  public synchronized void requestSystemCompaction(HRegion region, HStore store, String why)
+      throws IOException {
+    requestCompactionInternal(region, store, why, Store.NO_PRIORITY, false,
+      CompactionLifeCycleTracker.DUMMY, null);
+  }
+
+  private Optional<CompactionContext> selectCompaction(HRegion region, HStore store, int priority,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
+    Optional<CompactionContext> compaction = store.requestCompaction(priority, tracker, user);
+    if (!compaction.isPresent() && LOG.isDebugEnabled() && region.getRegionInfo() != null) {
+      LOG.debug("Not compacting " + region.getRegionInfo().getRegionNameAsString() +
+          " because compaction request was cancelled");
     }
     return compaction;
   }
@@ -468,33 +429,33 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
       if (cmp != 0) {
         return cmp;
       }
-      CompactionContext c1 = o1.compaction;
-      CompactionContext c2 = o2.compaction;
-      if (c1 == null) {
-        return c2 == null ? 0 : 1;
+      Optional<CompactionContext> c1 = o1.compaction;
+      Optional<CompactionContext> c2 = o2.compaction;
+      if (c1.isPresent()) {
+        return c2.isPresent() ? compare(c1.get().getRequest(), c2.get().getRequest()) : -1;
       } else {
-        return c2 == null ? -1 : compare(c1.getRequest(), c2.getRequest());
+        return c2.isPresent() ? 1 : 0;
       }
     }
   };
 
   private final class CompactionRunner implements Runnable {
-    private final Store store;
+    private final HStore store;
     private final HRegion region;
-    private CompactionContext compaction;
+    private final Optional<CompactionContext> compaction;
     private int queuedPriority;
     private ThreadPoolExecutor parent;
     private User user;
     private long time;
 
-    public CompactionRunner(Store store, Region region, CompactionContext compaction,
+    public CompactionRunner(HStore store, HRegion region, Optional<CompactionContext> compaction,
         ThreadPoolExecutor parent, User user) {
       super();
       this.store = store;
-      this.region = (HRegion) region;
+      this.region = region;
       this.compaction = compaction;
-      this.queuedPriority =
-          compaction == null ? store.getCompactPriority() : compaction.getRequest().getPriority();
+      this.queuedPriority = compaction.isPresent() ? compaction.get().getRequest().getPriority()
+          : store.getCompactPriority();
       this.parent = parent;
       this.user = user;
       this.time = System.currentTimeMillis();
@@ -502,14 +463,15 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
 
     @Override
     public String toString() {
-      return (this.compaction != null) ? ("Request = " + compaction.getRequest())
-          : ("regionName = " + region.toString() + ", storeName = " + store.toString() +
-             ", priority = " + queuedPriority + ", time = " + time);
+      return compaction.map(c -> "Request = " + c.getRequest())
+          .orElse("regionName = " + region.toString() + ", storeName = " + store.toString() +
+              ", priority = " + queuedPriority + ", time = " + time);
     }
 
     private void doCompaction(User user) {
+      CompactionContext c;
       // Common case - system compaction without a file selection. Select now.
-      if (this.compaction == null) {
+      if (!compaction.isPresent()) {
         int oldPriority = this.queuedPriority;
         this.queuedPriority = this.store.getCompactPriority();
         if (this.queuedPriority > oldPriority) {
@@ -518,44 +480,49 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
           this.parent.execute(this);
           return;
         }
+        Optional<CompactionContext> selected;
         try {
-          this.compaction = selectCompaction(this.region, this.store, queuedPriority, null, user);
+          selected = selectCompaction(this.region, this.store, queuedPriority,
+            CompactionLifeCycleTracker.DUMMY, user);
         } catch (IOException ex) {
           LOG.error("Compaction selection failed " + this, ex);
           server.checkFileSystem();
           region.decrementCompactionsQueuedCount();
           return;
         }
-        if (this.compaction == null) {
+        if (!selected.isPresent()) {
           region.decrementCompactionsQueuedCount();
           return; // nothing to do
         }
+        c = selected.get();
+        assert c.hasSelection();
         // Now see if we are in correct pool for the size; if not, go to the correct one.
         // We might end up waiting for a while, so cancel the selection.
-        assert this.compaction.hasSelection();
-        ThreadPoolExecutor pool = store.throttleCompaction(
-            compaction.getRequest().getSize()) ? longCompactions : shortCompactions;
+
+        ThreadPoolExecutor pool =
+            store.throttleCompaction(c.getRequest().getSize()) ? longCompactions : shortCompactions;
 
         // Long compaction pool can process small job
         // Short compaction pool should not process large job
         if (this.parent == shortCompactions && pool == longCompactions) {
-          this.store.cancelRequestedCompaction(this.compaction);
-          this.compaction = null;
+          this.store.cancelRequestedCompaction(c);
           this.parent = pool;
           this.parent.execute(this);
           return;
         }
+      } else {
+        c = compaction.get();
       }
       // Finally we can compact something.
-      assert this.compaction != null;
+      assert c != null;
 
-      this.compaction.getRequest().beforeExecute();
+      c.getRequest().getTracker().beforeExecute(store);
       try {
         // Note: please don't put single-compaction logic here;
         //       put it into region/store/etc. This is CST logic.
         long start = EnvironmentEdgeManager.currentTime();
         boolean completed =
-            region.compact(compaction, store, compactionThroughputController, user);
+            region.compact(c, store, compactionThroughputController, user);
         long now = EnvironmentEdgeManager.currentTime();
         LOG.info(((completed) ? "Completed" : "Aborted") + " compaction: " +
               this + "; duration=" + StringUtils.formatTimeDiff(now, start));
@@ -582,10 +549,10 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
         region.reportCompactionRequestFailure();
         server.checkFileSystem();
       } finally {
+        c.getRequest().getTracker().afterExecute(store);
         region.decrementCompactionsQueuedCount();
         LOG.debug("CompactSplitThread Status: " + CompactSplit.this);
       }
-      this.compaction.getRequest().afterExecute();
     }
 
     @Override
@@ -615,9 +582,9 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
     @Override
     public void rejectedExecution(Runnable runnable, ThreadPoolExecutor pool) {
       if (runnable instanceof CompactionRunner) {
-        CompactionRunner runner = (CompactionRunner)runnable;
+        CompactionRunner runner = (CompactionRunner) runnable;
         LOG.debug("Compaction Rejected: " + runner);
-        runner.store.cancelRequestedCompaction(runner.compaction);
+        runner.compaction.ifPresent(c -> runner.store.cancelRequestedCompaction(c));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
deleted file mode 100644
index d1f02fe..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Pair;
-
-@InterfaceAudience.Private
-public interface CompactionRequestor {
-  /**
-   * @param r Region to compact
-   * @param why Why compaction was requested -- used in debug messages
-   * @return The created {@link CompactionRequest CompactionRequests} or an empty list if no
-   *         compactions were started
-   * @throws IOException
-   */
-  List<CompactionRequest> requestCompaction(final Region r, final String why)
-      throws IOException;
-
-  /**
-   * @param r Region to compact
-   * @param why Why compaction was requested -- used in debug messages
-   * @param requests custom compaction requests. Each compaction must specify the store on which it
-   *          is acting. Can be <tt>null</tt> in which case a compaction will be attempted on all
-   *          stores for the region.
-   * @return The created {@link CompactionRequest CompactionRequests} or an empty list if no
-   *         compactions were started
-   * @throws IOException
-   */
-  List<CompactionRequest> requestCompaction(
-    final Region r, final String why, List<Pair<CompactionRequest, Store>> requests
-  )
-      throws IOException;
-
-  /**
-   * @param r Region to compact
-   * @param s Store within region to compact
-   * @param why Why compaction was requested -- used in debug messages
-   * @param request custom compaction request for the {@link Region} and {@link Store}. Custom
-   *          request must be <tt>null</tt> or be constructed with matching region and store.
-   * @return The created {@link CompactionRequest} or <tt>null</tt> if no compaction was started.
-   * @throws IOException
-   */
-  CompactionRequest requestCompaction(
-    final Region r, final Store s, final String why, CompactionRequest request
-  ) throws IOException;
-
-  /**
-   * @param r Region to compact
-   * @param why Why compaction was requested -- used in debug messages
-   * @param pri Priority of this compaction. minHeap. &lt;=0 is critical
-   * @param requests custom compaction requests. Each compaction must specify the store on which it
-   *          is acting. Can be <tt>null</tt> in which case a compaction will be attempted on all
-   *          stores for the region.
-   * @param user  the effective user
-   * @return The created {@link CompactionRequest CompactionRequests} or an empty list if no
-   *         compactions were started.
-   * @throws IOException
-   */
-  List<CompactionRequest> requestCompaction(
-    final Region r, final String why, int pri, List<Pair<CompactionRequest, Store>> requests,
-    User user
-  ) throws IOException;
-
-  /**
-   * @param r Region to compact
-   * @param s Store within region to compact
-   * @param why Why compaction was requested -- used in debug messages
-   * @param pri Priority of this compaction. minHeap. &lt;=0 is critical
-   * @param request custom compaction request to run. {@link Store} and {@link Region} for the
-   *          request must match the region and store specified here.
-   * @param user
-   * @return The created {@link CompactionRequest} or <tt>null</tt> if no compaction was started
-   * @throws IOException
-   */
-  CompactionRequest requestCompaction(
-    final Region r, final Store s, final String why, int pri, CompactionRequest request, User user
-  ) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
index b0eae71..e4476d0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllLargeStoresPolicy.java
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * enough, then all stores will be flushed.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy{
+public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy {
 
   private static final Log LOG = LogFactory.getLog(FlushAllLargeStoresPolicy.class);
 
@@ -48,20 +48,22 @@ public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy{
   }
 
   @Override
-  public Collection<Store> selectStoresToFlush() {
+  public Collection<HStore> selectStoresToFlush() {
     // no need to select stores if only one family
     if (region.getTableDescriptor().getColumnFamilyCount() == 1) {
       return region.stores.values();
     }
     // start selection
-    Collection<Store> stores = region.stores.values();
-    Set<Store> specificStoresToFlush = new HashSet<>();
-    for (Store store : stores) {
+    Collection<HStore> stores = region.stores.values();
+    Set<HStore> specificStoresToFlush = new HashSet<>();
+    for (HStore store : stores) {
       if (shouldFlush(store)) {
         specificStoresToFlush.add(store);
       }
     }
-    if (!specificStoresToFlush.isEmpty()) return specificStoresToFlush;
+    if (!specificStoresToFlush.isEmpty()) {
+      return specificStoresToFlush;
+    }
 
     // Didn't find any CFs which were above the threshold for selection.
     if (LOG.isDebugEnabled()) {
@@ -71,8 +73,8 @@ public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy{
   }
 
   @Override
-  protected boolean shouldFlush(Store store) {
-    return (super.shouldFlush(store) || region.shouldFlushStore(store));
+  protected boolean shouldFlush(HStore store) {
+    return super.shouldFlush(store) || region.shouldFlushStore(store);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllStoresPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllStoresPolicy.java
index 5c7b3af..97a04f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllStoresPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushAllStoresPolicy.java
@@ -28,7 +28,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 public class FlushAllStoresPolicy extends FlushPolicy {
 
   @Override
-  public Collection<Store> selectStoresToFlush() {
+  public Collection<HStore> selectStoresToFlush() {
     return region.stores.values();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
index e37a1a2..e0c6510 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
@@ -77,7 +77,7 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
     return flushSizeLowerBound;
   }
 
-  protected boolean shouldFlush(Store store) {
+  protected boolean shouldFlush(HStore store) {
     if (store.getSizeOfMemStore().getDataSize() > this.flushSizeLowerBound) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " +

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
index 1196bd5..c779ce3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java
@@ -32,26 +32,31 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class FlushNonSloppyStoresFirstPolicy extends FlushLargeStoresPolicy {
 
-  private Collection<Store> regularStores = new HashSet<>();
-  private Collection<Store> sloppyStores = new HashSet<>();
+  private Collection<HStore> regularStores = new HashSet<>();
+  private Collection<HStore> sloppyStores = new HashSet<>();
 
   /**
    * @return the stores need to be flushed.
    */
-  @Override public Collection<Store> selectStoresToFlush() {
-    Collection<Store> specificStoresToFlush = new HashSet<>();
-    for(Store store : regularStores) {
-      if(shouldFlush(store) || region.shouldFlushStore(store)) {
+  @Override
+  public Collection<HStore> selectStoresToFlush() {
+    Collection<HStore> specificStoresToFlush = new HashSet<>();
+    for (HStore store : regularStores) {
+      if (shouldFlush(store) || region.shouldFlushStore(store)) {
         specificStoresToFlush.add(store);
       }
     }
-    if(!specificStoresToFlush.isEmpty()) return specificStoresToFlush;
-    for(Store store : sloppyStores) {
-      if(shouldFlush(store)) {
+    if (!specificStoresToFlush.isEmpty()) {
+      return specificStoresToFlush;
+    }
+    for (HStore store : sloppyStores) {
+      if (shouldFlush(store)) {
         specificStoresToFlush.add(store);
       }
     }
-    if(!specificStoresToFlush.isEmpty()) return specificStoresToFlush;
+    if (!specificStoresToFlush.isEmpty()) {
+      return specificStoresToFlush;
+    }
     return region.stores.values();
   }
 
@@ -59,8 +64,8 @@ public class FlushNonSloppyStoresFirstPolicy extends FlushLargeStoresPolicy {
   protected void configureForRegion(HRegion region) {
     super.configureForRegion(region);
     this.flushSizeLowerBound = getFlushSizeLowerBound(region);
-    for(Store store : region.stores.values()) {
-      if(store.isSloppyMemstore()) {
+    for (HStore store : region.stores.values()) {
+      if (store.isSloppyMemstore()) {
         sloppyStores.add(store);
       } else {
         regularStores.add(store);

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicy.java
index bc49c92..fecbd2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushPolicy.java
@@ -44,6 +44,6 @@ public abstract class FlushPolicy extends Configured {
   /**
    * @return the stores need to be flushed.
    */
-  public abstract Collection<Store> selectStoresToFlush();
+  public abstract Collection<HStore> selectStoresToFlush();
 
 }


[03/47] hbase git commit: HBASE-18453 CompactionRequest should not be exposed to user directly

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 45b7d3c..4890f0d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -23,10 +23,12 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CompareOperator;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HDFSBlocksDistribution;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -41,13 +43,17 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.Service;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Service;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
 
 /**
  * Regions store data for a certain region of a table.  It stores all columns
@@ -105,7 +111,7 @@ public interface Region extends ConfigurationObserver {
    * <p>Use with caution.  Exposed for use of fixup utilities.
    * @return a list of the Stores managed by this region
    */
-  List<Store> getStores();
+  List<? extends Store> getStores();
 
   /**
    * Return the Store for the given family
@@ -115,7 +121,7 @@ public interface Region extends ConfigurationObserver {
   Store getStore(byte[] family);
 
   /** @return list of store file names for the given families */
-  List<String> getStoreFileList(byte [][] columns);
+  List<String> getStoreFileList(byte[][] columns);
 
   /**
    * Check the region's underlying store files, open the files that have not
@@ -753,6 +759,18 @@ public interface Region extends ConfigurationObserver {
    */
   CompactionState getCompactionState();
 
+  /**
+   * Request compaction on this region.
+   */
+  void requestCompaction(String why, int priority, CompactionLifeCycleTracker tracker, User user)
+      throws IOException;
+
+  /**
+   * Request compaction for the given family
+   */
+  void requestCompaction(byte[] family, String why, int priority,
+      CompactionLifeCycleTracker tracker, User user) throws IOException;
+
   /** Wait for all current flushes and compactions of the region to complete */
   void waitForFlushesAndCompactions();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 bed2a7a..f282766 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
@@ -74,6 +74,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
 import org.apache.hadoop.hbase.security.User;
@@ -499,18 +500,18 @@ public class RegionCoprocessorHost
   /**
    * See
    * {@link RegionObserver#preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
-   *   InternalScanner, CompactionRequest, long)}
+   *   InternalScanner, CompactionLifeCycleTracker, long)}
    */
-  public InternalScanner preCompactScannerOpen(final Store store,
-      final List<StoreFileScanner> scanners, final ScanType scanType, final long earliestPutTs,
-      final CompactionRequest request, final User user, final long readPoint) throws IOException {
+  public InternalScanner preCompactScannerOpen(Store store, List<StoreFileScanner> scanners,
+      ScanType scanType, long earliestPutTs, CompactionLifeCycleTracker tracker, User user,
+      long readPoint) throws IOException {
     return execOperationWithResult(null,
         coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>(user) {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
         setResult(oserver.preCompactScannerOpen(ctx, store, scanners, scanType,
-          earliestPutTs, getResult(), request, readPoint));
+          earliestPutTs, getResult(), tracker, readPoint));
       }
     });
   }
@@ -520,17 +521,17 @@ public class RegionCoprocessorHost
    * available candidates.
    * @param store The store where compaction is being requested
    * @param candidates The currently available store files
-   * @param request custom compaction request
+   * @param tracker used to track the life cycle of a compaction
    * @return If {@code true}, skip the normal selection process and use the current list
    * @throws IOException
    */
-  public boolean preCompactSelection(final Store store, final List<StoreFile> candidates,
-      final CompactionRequest request, final User user) throws IOException {
+  public boolean preCompactSelection(Store store, List<StoreFile> candidates,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preCompactSelection(ctx, store, candidates, request);
+        oserver.preCompactSelection(ctx, store, candidates, tracker);
       }
     });
   }
@@ -540,21 +541,17 @@ public class RegionCoprocessorHost
    * candidates.
    * @param store The store where compaction is being requested
    * @param selected The store files selected to compact
-   * @param request custom compaction
+   * @param tracker used to track the life cycle of a compaction
    */
-  public void postCompactSelection(final Store store, final ImmutableList<StoreFile> selected,
-      final CompactionRequest request, final User user) {
-    try {
-      execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
-        @Override
-        public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-            throws IOException {
-          oserver.postCompactSelection(ctx, store, selected, request);
-        }
-      });
-    } catch (IOException e) {
-      LOG.warn(e);
-    }
+  public void postCompactSelection(Store store, ImmutableList<StoreFile> selected,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
+      @Override
+      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
+          throws IOException {
+        oserver.postCompactSelection(ctx, store, selected, tracker);
+      }
+    });
   }
 
   /**
@@ -562,18 +559,17 @@ public class RegionCoprocessorHost
    * @param store the store being compacted
    * @param scanner the scanner used to read store data during compaction
    * @param scanType type of Scan
-   * @param request the compaction that will be executed
+   * @param tracker used to track the life cycle of a compaction
    * @throws IOException
    */
-  public InternalScanner preCompact(final Store store, final InternalScanner scanner,
-      final ScanType scanType, final CompactionRequest request, final User user)
-      throws IOException {
+  public InternalScanner preCompact(Store store, InternalScanner scanner, ScanType scanType,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
     return execOperationWithResult(false, scanner,
         coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>(user) {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
-        setResult(oserver.preCompact(ctx, store, getResult(), scanType, request));
+        setResult(oserver.preCompact(ctx, store, getResult(), scanType, tracker));
       }
     });
   }
@@ -582,16 +578,16 @@ public class RegionCoprocessorHost
    * Called after the store compaction has completed.
    * @param store the store being compacted
    * @param resultFile the new store file written during compaction
-   * @param request the compaction that is being executed
+   * @param tracker used to track the life cycle of a compaction
    * @throws IOException
    */
-  public void postCompact(final Store store, final StoreFile resultFile,
-      final CompactionRequest request, final User user) throws IOException {
+  public void postCompact(Store store, StoreFile resultFile, CompactionLifeCycleTracker tracker,
+      User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postCompact(ctx, store, resultFile, request);
+        oserver.postCompact(ctx, store, resultFile, tracker);
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 b56c925..209fce8 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
@@ -64,11 +64,6 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
   List<WAL> getWALs() throws IOException;
 
   /**
-   * @return Implementation of {@link CompactionRequestor} or null.
-   */
-  CompactionRequestor getCompactionRequester();
-
-  /**
    * @return Implementation of {@link FlushRequester} or null.
    */
   FlushRequester getFlushRequester();

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
index 7281626..5ccd6e3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
@@ -76,7 +76,7 @@ public abstract class RegionSplitPolicy extends Configured {
     if (explicitSplitPoint != null) {
       return explicitSplitPoint;
     }
-    List<Store> stores = region.getStores();
+    List<HStore> stores = region.getStores();
 
     byte[] splitPointFromLargestStore = null;
     long largestStoreSize = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index 4bb31ae..b7e83bf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.Optional;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.CellComparator;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
@@ -51,7 +53,8 @@ import org.apache.hadoop.hbase.security.User;
 @InterfaceStability.Evolving
 public interface Store extends HeapSize, StoreConfigInformation, PropagatingConfigurationObserver {
 
-  /* The default priority for user-specified compaction requests.
+  /**
+   * The default priority for user-specified compaction requests.
    * The user gets top priority unless we have blocking compactions. (Pri <= 0)
    */
   int PRIORITY_USER = 1;
@@ -253,17 +256,12 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
    */
   CompactionProgress getCompactionProgress();
 
-  CompactionContext requestCompaction() throws IOException;
-
-  /**
-   * @deprecated see requestCompaction(int, CompactionRequest, User)
-   */
-  @Deprecated
-  CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
-      throws IOException;
+  default Optional<CompactionContext> requestCompaction() throws IOException {
+    return requestCompaction(NO_PRIORITY, CompactionLifeCycleTracker.DUMMY, null);
+  }
 
-  CompactionContext requestCompaction(int priority, CompactionRequest baseRequest, User user)
-      throws IOException;
+  Optional<CompactionContext> requestCompaction(int priority, CompactionLifeCycleTracker tracker,
+      User user) throws IOException;
 
   void cancelRequestedCompaction(CompactionContext compaction);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionLifeCycleTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionLifeCycleTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionLifeCycleTracker.java
new file mode 100644
index 0000000..38fec7e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionLifeCycleTracker.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.compactions;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * Used to track compaction execution.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface CompactionLifeCycleTracker {
+
+  static CompactionLifeCycleTracker DUMMY = new CompactionLifeCycleTracker() {
+  };
+
+  /**
+   * Called before compaction is executed by CompactSplitThread.
+   * <p>
+   * Requesting compaction on a region can lead to multiple compactions on different stores, so we
+   * will pass the {@link Store} in to tell you the store we operate on.
+   */
+  default void beforeExecute(Store store) {
+  }
+
+  /**
+   * Called after compaction is executed by CompactSplitThread.
+   * <p>
+   * Requesting compaction on a region can lead to multiple compactions on different stores, so we
+   * will pass the {@link Store} in to tell you the store we operate on.
+   */
+  default void afterExecute(Store store) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
index 69e354b..e05c165 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
@@ -18,25 +18,21 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.stream.Collectors;
 
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 
 /**
  * This class holds all logical details necessary to run a compaction.
  */
-@InterfaceAudience.LimitedPrivate({ "coprocessor" })
-@InterfaceStability.Evolving
+@InterfaceAudience.Private
 public class CompactionRequest {
 
   // was this compaction promoted to an off-peak
@@ -53,56 +49,18 @@ public class CompactionRequest {
   private String regionName = "";
   private String storeName = "";
   private long totalSize = -1L;
+  private CompactionLifeCycleTracker tracker = CompactionLifeCycleTracker.DUMMY;
 
-  /**
-   * This ctor should be used by coprocessors that want to subclass CompactionRequest.
-   */
-  public CompactionRequest() {
+  public CompactionRequest(Collection<StoreFile> files) {
     this.selectionTime = EnvironmentEdgeManager.currentTime();
     this.timeInNanos = System.nanoTime();
-  }
-
-  public CompactionRequest(Collection<StoreFile> files) {
-    this();
-    Preconditions.checkNotNull(files);
-    this.filesToCompact = files;
+    this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
     recalculateSize();
   }
 
   public void updateFiles(Collection<StoreFile> files) {
-    this.filesToCompact = files;
-    recalculateSize();
-  }
-
-  /**
-   * Called before compaction is executed by CompactSplitThread; for use by coproc subclasses.
-   */
-  public void beforeExecute() {}
-
-  /**
-   * Called after compaction is executed by CompactSplitThread; for use by coproc subclasses.
-   */
-  public void afterExecute() {}
-
-  /**
-   * Combines the request with other request. Coprocessors subclassing CR may override
-   * this if they want to do clever things based on CompactionPolicy selection that
-   * is passed to this method via "other". The default implementation just does a copy.
-   * @param other Request to combine with.
-   * @return The result (may be "this" or "other").
-   */
-  public CompactionRequest combineWith(CompactionRequest other) {
-    this.filesToCompact = new ArrayList<>(other.getFiles());
-    this.isOffPeak = other.isOffPeak;
-    this.isMajor = other.isMajor;
-    this.priority = other.priority;
-    this.selectionTime = other.selectionTime;
-    this.timeInNanos = other.timeInNanos;
-    this.regionName = other.regionName;
-    this.storeName = other.storeName;
-    this.totalSize = other.totalSize;
+    this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
     recalculateSize();
-    return this;
   }
 
   public Collection<StoreFile> getFiles() {
@@ -168,6 +126,14 @@ public class CompactionRequest {
         : (isMajor ? DisplayCompactionType.MAJOR : DisplayCompactionType.ALL_FILES);
   }
 
+  public void setTracker(CompactionLifeCycleTracker tracker) {
+    this.tracker = tracker;
+  }
+
+  public CompactionLifeCycleTracker getTracker() {
+    return tracker;
+  }
+
   @Override
   public String toString() {
     String fsList = filesToCompact.stream().filter(f -> f.getReader() != null)
@@ -186,12 +152,7 @@ public class CompactionRequest {
    * @param files files that should be included in the compaction
    */
   private void recalculateSize() {
-    long sz = 0;
-    for (StoreFile sf : this.filesToCompact) {
-      StoreFileReader r = sf.getReader();
-      sz += r == null ? 0 : r.length();
-    }
-    this.totalSize = sz;
+    this.totalSize = filesToCompact.stream().map(StoreFile::getReader)
+        .mapToLong(r -> r != null ? r.length() : 0L).sum();
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index 0ca925e..e6d1935 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@ -338,14 +338,14 @@ public abstract class Compactor<T extends CellSink> {
    * @param readPoint the read point to help create scanner by Coprocessor if required.
    * @return Scanner override by coprocessor; null if not overriding.
    */
-  protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
-      final ScanType scanType, final long earliestPutTs, final List<StoreFileScanner> scanners,
-      User user, final long readPoint) throws IOException {
+  protected InternalScanner preCreateCoprocScanner(CompactionRequest request, ScanType scanType,
+      long earliestPutTs, List<StoreFileScanner> scanners, User user, long readPoint)
+      throws IOException {
     if (store.getCoprocessorHost() == null) {
       return null;
     }
     return store.getCoprocessorHost().preCompactScannerOpen(store, scanners, scanType,
-        earliestPutTs, request, user, readPoint);
+      earliestPutTs, request.getTracker(), user, readPoint);
   }
 
   /**
@@ -355,12 +355,13 @@ public abstract class Compactor<T extends CellSink> {
    * @param scanner The default scanner created for compaction.
    * @return Scanner scanner to use (usually the default); null if compaction should not proceed.
    */
-  protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
-      final ScanType scanType, final InternalScanner scanner, User user) throws IOException {
+  protected InternalScanner postCreateCoprocScanner(CompactionRequest request, ScanType scanType,
+      InternalScanner scanner, User user) throws IOException {
     if (store.getCoprocessorHost() == null) {
       return scanner;
     }
-    return store.getCoprocessorHost().preCompact(store, scanner, scanType, request, user);
+    return store.getCoprocessorHost().preCompact(store, scanner, scanType, request.getTracker(),
+      user);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 f7d70d2..c34fc6d 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
@@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
@@ -1532,7 +1533,8 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException {
+      InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker)
+      throws IOException {
     requirePermission(getActiveUser(c), "compact", getTableName(c.getEnvironment()), null, null,
       Action.ADMIN, Action.CREATE);
     return scanner;

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
index c1e5aae..04aafa7 100644
--- a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
@@ -92,7 +92,7 @@
     </div>
 
 <% if(region != null) { //
-     List<Store> stores = region.getStores();
+     List<? extends Store> stores = region.getStores();
      for (Store store : stores) {
        String cf = store.getColumnFamilyName();
        Collection<StoreFile> storeFiles = store.getStorefiles(); %>

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 a99345b..76580f1 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase;
 
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Collections;
@@ -36,11 +38,8 @@ 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.ipc.RpcServerInterface;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
-import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
 import org.apache.hadoop.hbase.regionserver.Leases;
@@ -51,14 +50,14 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
 import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.protobuf.Service;
-
 /**
  * Basic mock region server services.  Should only be instantiated by HBaseTestingUtility.b
  */
@@ -160,11 +159,6 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public CompactionRequestor getCompactionRequester() {
-    return null;
-  }
-
-  @Override
   public ClusterConnection getConnection() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index 94d1cf6..85f65e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -124,7 +124,7 @@ public class TestIOFencing {
     }
 
     @Override
-    public boolean compact(CompactionContext compaction, Store store,
+    public boolean compact(CompactionContext compaction, HStore store,
         ThroughputController throughputController) throws IOException {
       try {
         return super.compact(compaction, store, throughputController);
@@ -134,7 +134,7 @@ public class TestIOFencing {
     }
 
     @Override
-    public boolean compact(CompactionContext compaction, Store store,
+    public boolean compact(CompactionContext compaction, HStore store,
         ThroughputController throughputController, User user) throws IOException {
       try {
         return super.compact(compaction, store, throughputController, user);

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 aff3d99..ac404bb 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
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -256,7 +256,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
     @Override
     public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
         Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-        InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
+        InternalScanner s, CompactionLifeCycleTracker request, long readPoint) throws IOException {
       return createCompactorScanner(store, scanners, scanType, earliestPutTs);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 d5f3358..08ec09b 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
@@ -784,7 +784,7 @@ public class TestBlockEvictionFromClient {
   }
 
   private BlockCache setCacheProperties(Region region) {
-    Iterator<Store> strItr = region.getStores().iterator();
+    Iterator<? extends Store> strItr = region.getStores().iterator();
     BlockCache cache = null;
     while (strItr.hasNext()) {
       Store store = strItr.next();

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
index bef6f6b..023f8fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -203,20 +204,20 @@ public class SimpleRegionObserver implements RegionObserver {
 
   @Override
   public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      List<StoreFile> candidates, CompactionRequest request) throws IOException {
+      List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
     ctPreCompactSelect.incrementAndGet();
   }
 
   @Override
   public void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      ImmutableList<StoreFile> selected, CompactionRequest request) {
+      ImmutableList<StoreFile> selected, CompactionLifeCycleTracker tracker) {
     ctPostCompactSelect.incrementAndGet();
   }
 
-
   @Override
   public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException {
+      InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker)
+      throws IOException {
     ctPreCompact.incrementAndGet();
     return scanner;
   }
@@ -224,14 +225,14 @@ public class SimpleRegionObserver implements RegionObserver {
   @Override
   public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
       Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
+      InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
     ctPreCompactScanner.incrementAndGet();
     return s;
   }
 
   @Override
   public void postCompact(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      StoreFile resultFile, CompactionRequest request) throws IOException {
+      StoreFile resultFile, CompactionLifeCycleTracker tracker) throws IOException {
     ctPostCompact.incrementAndGet();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 97c45ab..5cf0bb3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -194,13 +195,13 @@ public class TestCoprocessorInterface {
     }
     @Override
     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-        Store store, InternalScanner scanner, ScanType scanType, CompactionRequest request) {
+        Store store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker) {
       preCompactCalled = true;
       return scanner;
     }
     @Override
     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-        Store store, StoreFile resultFile, CompactionRequest request) {
+        Store store, StoreFile resultFile, CompactionLifeCycleTracker tracker) {
       postCompactCalled = true;
     }
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
index 07fc179..0641b56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
@@ -71,7 +71,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
@@ -417,7 +417,7 @@ public class TestRegionObserverInterface {
 
     @Override
     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
-        final InternalScanner scanner, final ScanType scanType, CompactionRequest request) {
+        InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker) {
       return new InternalScanner() {
         @Override
         public boolean next(List<Cell> results) throws IOException {
@@ -456,7 +456,7 @@ public class TestRegionObserverInterface {
 
     @Override
     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
-        StoreFile resultFile, CompactionRequest request) {
+        StoreFile resultFile, CompactionLifeCycleTracker tracker) {
       lastCompaction = EnvironmentEdgeManager.currentTime();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index 8ab0175..9c06c3e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
@@ -61,6 +62,7 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
@@ -155,7 +157,7 @@ public class TestRegionObserverScannerOpenHook {
     @Override
     public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
         Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
-        long earliestPutTs, InternalScanner s, CompactionRequest request, long readPoint)
+        long earliestPutTs, InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint)
         throws IOException {
       scanners.forEach(KeyValueScanner::close);
       return NO_DATA;
@@ -252,7 +254,7 @@ public class TestRegionObserverScannerOpenHook {
     }
 
     @Override
-    public boolean compact(CompactionContext compaction, Store store,
+    public boolean compact(CompactionContext compaction, HStore store,
         ThroughputController throughputController) throws IOException {
       boolean ret = super.compact(compaction, store, throughputController);
       if (ret) compactionStateChangeLatch.countDown();
@@ -260,7 +262,7 @@ public class TestRegionObserverScannerOpenHook {
     }
 
     @Override
-    public boolean compact(CompactionContext compaction, Store store,
+    public boolean compact(CompactionContext compaction, HStore store,
         ThroughputController throughputController, User user) throws IOException {
       boolean ret = super.compact(compaction, store, throughputController, user);
       if (ret) compactionStateChangeLatch.countDown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 3ffa61b..ba18299 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
@@ -47,6 +47,27 @@ import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
+import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
+import org.apache.hadoop.hbase.regionserver.FlushRequester;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
+import org.apache.hadoop.hbase.regionserver.Leases;
+import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
+import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
@@ -61,10 +82,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegion
 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.GetRegionLoadRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse;
 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;
@@ -102,27 +123,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRespon
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
-import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
-import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
-import org.apache.hadoop.hbase.regionserver.FlushRequester;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
-import org.apache.hadoop.hbase.regionserver.Leases;
-import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
-import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
-import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * A mock RegionServer implementation.
@@ -315,12 +315,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public CompactionRequestor getCompactionRequester() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public FlushRequester getFlushRequester() {
     // TODO Auto-generated method stub
     return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
index 398d14d..12aed50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
@@ -86,7 +86,7 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -716,14 +716,14 @@ public class TestMobCompactor {
   }
 
   /**
-   * This copro overwrites the default compaction policy. It always chooses two latest
-   * hfiles and compacts them into a new one.
+   * This copro overwrites the default compaction policy. It always chooses two latest hfiles and
+   * compacts them into a new one.
    */
   public static class CompactTwoLatestHfilesCopro implements RegionObserver {
+
     @Override
-    public void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Store store, final List<StoreFile> candidates, final CompactionRequest request)
-      throws IOException {
+    public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+        List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
 
       int count = candidates.size();
       if (count >= 2) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 5179b84..8015115 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -461,7 +462,7 @@ public class TestNamespaceAuditor {
 
     @Override
     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
-        StoreFile resultFile, CompactionRequest request) throws IOException {
+        StoreFile resultFile, CompactionLifeCycleTracker tracker) throws IOException {
       postCompact.countDown();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
index 823b1f7..0d07e1a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java
@@ -36,9 +36,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -324,7 +324,7 @@ public class TestFileSystemUtilizationChore {
     final HRegionInfo info = mock(HRegionInfo.class);
     when(r.getRegionInfo()).thenReturn(info);
     List<Store> stores = new ArrayList<>();
-    when(r.getStores()).thenReturn(stores);
+    when(r.getStores()).thenReturn((List) stores);
     for (Long storeSize : storeSizes) {
       final Store s = mock(Store.class);
       stores.add(s);
@@ -338,7 +338,7 @@ public class TestFileSystemUtilizationChore {
     final HRegionInfo info = mock(HRegionInfo.class);
     when(r.getRegionInfo()).thenReturn(info);
     List<Store> stores = new ArrayList<>();
-    when(r.getStores()).thenReturn(stores);
+    when(r.getStores()).thenReturn((List) stores);
     assertEquals(
         "Logic error, storeSizes and linkSizes must be equal in size", storeSizes.size(),
         hfileSizes.size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
index fcc3d4a..36c2e19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.client.TestFromClientSideWithCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 
 /**
@@ -61,7 +62,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
   public InternalScanner preCompactScannerOpen(
       final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
       List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
+      InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
     // this demonstrates how to override the scanners default behavior
     ScanInfo oldSI = store.getScanInfo();
     ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/StatefulStoreMockMaker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/StatefulStoreMockMaker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/StatefulStoreMockMaker.java
index 0526462..a1fe87b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/StatefulStoreMockMaker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/StatefulStoreMockMaker.java
@@ -18,11 +18,16 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.mockito.Matchers.*;
-import static org.mockito.Mockito.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Optional;
 
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.security.User;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -33,15 +38,23 @@ import org.mockito.stubbing.Answer;
  */
 public class StatefulStoreMockMaker {
   // Add and expand the methods and answers as needed.
-  public CompactionContext selectCompaction() { return null; }
-  public void cancelCompaction(Object originalContext) {}
-  public int getPriority() { return 0; }
+  public Optional<CompactionContext> selectCompaction() {
+    return Optional.empty();
+  }
 
-  private class SelectAnswer implements Answer<CompactionContext> {
-    public CompactionContext answer(InvocationOnMock invocation) throws Throwable {
+  public void cancelCompaction(Object originalContext) {
+  }
+
+  public int getPriority() {
+    return 0;
+  }
+
+  private class SelectAnswer implements Answer<Optional<CompactionContext>> {
+    public Optional<CompactionContext> answer(InvocationOnMock invocation) throws Throwable {
       return selectCompaction();
     }
   }
+
   private class PriorityAnswer implements Answer<Integer> {
     public Integer answer(InvocationOnMock invocation) throws Throwable {
       return getPriority();
@@ -53,15 +66,13 @@ public class StatefulStoreMockMaker {
     }
   }
 
-  public Store createStoreMock(String name) throws Exception {
-    Store store = mock(Store.class, name);
-    when(store.requestCompaction(
-        anyInt(), isNull(CompactionRequest.class))).then(new SelectAnswer());
-    when(store.requestCompaction(
-      anyInt(), isNull(CompactionRequest.class), any(User.class))).then(new SelectAnswer());
+  public HStore createStoreMock(String name) throws Exception {
+    HStore store = mock(HStore.class, name);
+    when(store.requestCompaction(anyInt(), any(CompactionLifeCycleTracker.class), any(User.class)))
+        .then(new SelectAnswer());
     when(store.getCompactPriority()).then(new PriorityAnswer());
-    doAnswer(new CancelAnswer()).when(
-        store).cancelRequestedCompaction(any(CompactionContext.class));
+    doAnswer(new CancelAnswer()).when(store)
+        .cancelRequestedCompaction(any(CompactionContext.class));
     return store;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index be078f2..3649823 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -33,8 +33,8 @@ import static org.mockito.Mockito.when;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
@@ -65,7 +66,6 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
@@ -298,15 +298,16 @@ public class TestCompaction {
     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
 
     // setup a region/store with some files
-    Store store = r.getStore(COLUMN_FAMILY);
+    HStore store = r.getStore(COLUMN_FAMILY);
     createStoreFile(r);
     for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
       createStoreFile(r);
     }
 
     CountDownLatch latch = new CountDownLatch(1);
-    TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
-    thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, request,null);
+    Tracker tracker = new Tracker(latch);
+    thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, tracker,
+      null);
     // wait for the latch to complete.
     latch.await();
 
@@ -322,7 +323,7 @@ public class TestCompaction {
     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
 
     // setup a region/store with some files
-    Store store = r.getStore(COLUMN_FAMILY);
+    HStore store = r.getStore(COLUMN_FAMILY);
     createStoreFile(r);
     for (int i = 0; i < HStore.DEFAULT_BLOCKING_STOREFILE_COUNT - 1; i++) {
       createStoreFile(r);
@@ -337,9 +338,9 @@ public class TestCompaction {
     long preFailedCount = metricsWrapper.getNumCompactionsFailed();
 
     CountDownLatch latch = new CountDownLatch(1);
-    TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
+    Tracker tracker = new Tracker(latch);
     thread.requestCompaction(mockRegion, store, "test custom comapction", Store.PRIORITY_USER,
-        request, null);
+      tracker, null);
     // wait for the latch to complete.
     latch.await(120, TimeUnit.SECONDS);
 
@@ -370,20 +371,17 @@ public class TestCompaction {
 
     // setup a region/store with some files
     int numStores = r.getStores().size();
-    List<Pair<CompactionRequest, Store>> requests = new ArrayList<>(numStores);
     CountDownLatch latch = new CountDownLatch(numStores);
+    Tracker tracker = new Tracker(latch);
     // create some store files and setup requests for each store on which we want to do a
     // compaction
-    for (Store store : r.getStores()) {
+    for (HStore store : r.getStores()) {
       createStoreFile(r, store.getColumnFamilyName());
       createStoreFile(r, store.getColumnFamilyName());
       createStoreFile(r, store.getColumnFamilyName());
-      requests.add(new Pair<>(new TrackableCompactionRequest(latch), store));
+      thread.requestCompaction(r, store, "test mulitple custom comapctions", Store.PRIORITY_USER,
+        tracker, null);
     }
-
-    thread.requestCompaction(r, "test mulitple custom comapctions", Store.PRIORITY_USER,
-      Collections.unmodifiableList(requests), null);
-
     // wait for the latch to complete.
     latch.await();
 
@@ -428,7 +426,7 @@ public class TestCompaction {
     }
 
     @Override
-    public synchronized CompactionContext selectCompaction() {
+    public synchronized Optional<CompactionContext> selectCompaction() {
       CompactionContext ctx = new TestCompactionContext(new ArrayList<>(notCompacting));
       compacting.addAll(notCompacting);
       notCompacting.clear();
@@ -437,7 +435,7 @@ public class TestCompaction {
       } catch (IOException ex) {
         fail("Shouldn't happen");
       }
-      return ctx;
+      return Optional.of(ctx);
     }
 
     @Override
@@ -499,14 +497,14 @@ public class TestCompaction {
     }
 
     @Override
-    public CompactionContext selectCompaction() {
+    public Optional<CompactionContext> selectCompaction() {
       this.blocked = new BlockingCompactionContext();
       try {
         this.blocked.select(null, false, false, false);
       } catch (IOException ex) {
         fail("Shouldn't happen");
       }
-      return this.blocked;
+      return Optional.of(blocked);
     }
 
     @Override
@@ -527,13 +525,13 @@ public class TestCompaction {
     }
 
     @Override
-    public Store createStoreMock(String name) throws Exception {
+    public HStore createStoreMock(String name) throws Exception {
       return createStoreMock(Integer.MIN_VALUE, name);
     }
 
-    public Store createStoreMock(int priority, String name) throws Exception {
+    public HStore createStoreMock(int priority, String name) throws Exception {
       // Override the mock to always return the specified priority.
-      Store s = super.createStoreMock(name);
+      HStore s = super.createStoreMock(name);
       when(s.getCompactPriority()).thenReturn(priority);
       return s;
     }
@@ -555,7 +553,7 @@ public class TestCompaction {
     // Set up the region mock that redirects compactions.
     HRegion r = mock(HRegion.class);
     when(
-      r.compact(any(CompactionContext.class), any(Store.class),
+      r.compact(any(CompactionContext.class), any(HStore.class),
         any(ThroughputController.class), any(User.class))).then(new Answer<Boolean>() {
       @Override
       public Boolean answer(InvocationOnMock invocation) throws Throwable {
@@ -568,7 +566,7 @@ public class TestCompaction {
     // Set up store mocks for 2 "real" stores and the one we use for blocking CST.
     ArrayList<Integer> results = new ArrayList<>();
     StoreMockMaker sm = new StoreMockMaker(results), sm2 = new StoreMockMaker(results);
-    Store store = sm.createStoreMock("store1"), store2 = sm2.createStoreMock("store2");
+    HStore store = sm.createStoreMock("store1"), store2 = sm2.createStoreMock("store2");
     BlockingStoreMockMaker blocker = new BlockingStoreMockMaker();
 
     // First, block the compaction thread so that we could muck with queue.
@@ -691,24 +689,20 @@ public class TestCompaction {
   }
 
   /**
-   * Simple {@link CompactionRequest} on which you can wait until the requested compaction finishes.
+   * Simple {@link CompactionLifeCycleTracker} on which you can wait until the requested compaction
+   * finishes.
    */
-  public static class TrackableCompactionRequest extends CompactionRequest {
-    private CountDownLatch done;
-
-    /**
-     * Constructor for a custom compaction. Uses the setXXX methods to update the state of the
-     * compaction before being used.
-     */
-    public TrackableCompactionRequest(CountDownLatch finished) {
-      super();
-      this.done = finished;
+  public static class Tracker implements CompactionLifeCycleTracker {
+
+    private final CountDownLatch done;
+
+    public Tracker(CountDownLatch done) {
+      this.done = done;
     }
 
     @Override
-    public void afterExecute() {
-      super.afterExecute();
-      this.done.countDown();
+    public void afterExecute(Store store) {
+      done.countDown();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 2cea121..603203a 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
@@ -28,6 +28,7 @@ import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentSkipListSet;
 
 import javax.crypto.spec.SecretKeySpec;
@@ -60,6 +61,7 @@ import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
@@ -536,8 +538,9 @@ public class TestHMobStore {
 
     // Trigger major compaction
     this.store.triggerMajorCompaction();
-    CompactionContext requestCompaction = this.store.requestCompaction(1, null);
-    this.store.compact(requestCompaction, NoLimitThroughputController.INSTANCE, null);
+    Optional<CompactionContext> requestCompaction =
+        this.store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null);
+    this.store.compact(requestCompaction.get(), NoLimitThroughputController.INSTANCE, null);
     Assert.assertEquals(1, this.store.getStorefiles().size());
 
     //Check encryption after compaction

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index ea14962..97f8ce3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -67,6 +67,7 @@ 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.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -254,7 +255,8 @@ public class TestHRegionServerBulkLoad {
     static int sleepDuration;
     @Override
     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
-        InternalScanner scanner, ScanType scanType, CompactionRequest request) throws IOException {
+        InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker)
+        throws IOException {
       try {
         Thread.sleep(sleepDuration);
       } catch (InterruptedException ie) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
index f45c76c..707540a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
@@ -417,7 +418,7 @@ public class TestMajorCompaction {
     }
     store.triggerMajorCompaction();
 
-    CompactionRequest request = store.requestCompaction(Store.NO_PRIORITY, null).getRequest();
+    CompactionRequest request = store.requestCompaction().get().getRequest();
     assertNotNull("Expected to receive a compaction request", request);
     assertEquals(
       "System-requested major compaction should not occur if there are too many store files",
@@ -436,7 +437,9 @@ public class TestMajorCompaction {
       createStoreFile(r);
     }
     store.triggerMajorCompaction();
-    CompactionRequest request = store.requestCompaction(Store.PRIORITY_USER, null).getRequest();
+    CompactionRequest request =
+        store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null).get()
+            .getRequest();
     assertNotNull("Expected to receive a compaction request", request);
     assertEquals(
       "User-requested major compaction should always occur, even if there are too many store files",

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 69bc9a7..3ba2299 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -31,6 +31,7 @@ import java.io.InterruptedIOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -70,7 +71,11 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
-import org.apache.hadoop.hbase.master.*;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.MasterRpcServices;
+import org.apache.hadoop.hbase.master.NoSuchProcedureException;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
@@ -234,7 +239,7 @@ public class TestSplitTransactionOnCluster {
     assertEquals(1, cluster.getRegions(tableName).size());
 
     HRegion region = cluster.getRegions(tableName).get(0);
-    Store store = region.getStore(cf);
+    HStore store = region.getStore(cf);
     int regionServerIndex = cluster.getServerWith(region.getRegionInfo().getRegionName());
     HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
 
@@ -246,8 +251,8 @@ public class TestSplitTransactionOnCluster {
     int fileNum = store.getStorefiles().size();
     // 0, Compaction Request
     store.triggerMajorCompaction();
-    CompactionContext cc = store.requestCompaction();
-    assertNotNull(cc);
+    Optional<CompactionContext> cc = store.requestCompaction();
+    assertTrue(cc.isPresent());
     // 1, A timeout split
     // 1.1 close region
     assertEquals(2, region.close(false).get(cf).size());
@@ -255,7 +260,7 @@ public class TestSplitTransactionOnCluster {
     region.initialize();
 
     // 2, Run Compaction cc
-    assertFalse(region.compact(cc, store, NoLimitThroughputController.INSTANCE));
+    assertFalse(region.compact(cc.get(), store, NoLimitThroughputController.INSTANCE));
     assertTrue(fileNum > store.getStorefiles().size());
 
     // 3, Split

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
index c959a22..78c5330 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
@@ -117,7 +117,7 @@ public class TestSplitWalDataLoss {
       }
     }).when(spiedRegion).internalFlushCacheAndCommit(Matchers.<WAL> any(),
       Matchers.<MonitoredTask> any(), Matchers.<PrepareFlushResult> any(),
-      Matchers.<Collection<Store>> any());
+      Matchers.<Collection<HStore>> any());
     // Find region key; don't pick up key for hbase:meta by mistake.
     String key = null;
     for (Map.Entry<String, Region> entry: rs.onlineRegions.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 3674303..2095dcd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.spy;
@@ -47,7 +46,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Consumer;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -74,6 +72,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -105,12 +106,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.mockito.Mockito;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 /**
  * Test class for the Store
  */
@@ -371,7 +366,7 @@ public class TestStore {
      // There will be no compaction due to threshold above. Last file will not be replaced.
     for (int i = 1; i <= storeFileNum - 1; i++) {
       // verify the expired store file.
-      assertNull(this.store.requestCompaction());
+      assertFalse(this.store.requestCompaction().isPresent());
       Collection<StoreFile> sfs = this.store.getStorefiles();
       // Ensure i files are gone.
       if (minVersions == 0) {
@@ -386,7 +381,7 @@ public class TestStore {
       // Let the next store file expired.
       edge.incrementTime(sleepTime);
     }
-    assertNull(this.store.requestCompaction());
+    assertFalse(this.store.requestCompaction().isPresent());
 
     Collection<StoreFile> sfs = this.store.getStorefiles();
     // Assert the last expired file is not removed.
@@ -422,7 +417,7 @@ public class TestStore {
     Assert.assertEquals(lowestTimeStampFromManager,lowestTimeStampFromFS);
 
     // after compact; check the lowest time stamp
-    store.compact(store.requestCompaction(), NoLimitThroughputController.INSTANCE, null);
+    store.compact(store.requestCompaction().get(), NoLimitThroughputController.INSTANCE, null);
     lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());
     lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());
     Assert.assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS);

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index 54073bc..d25829d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -823,12 +823,12 @@ public abstract class AbstractTestWALReplay {
           final HRegion region =
               new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) {
             @Override
-            protected FlushResult internalFlushcache(final WAL wal, final long myseqid,
-                final Collection<Store> storesToFlush, MonitoredTask status,
+            protected FlushResultImpl internalFlushcache(final WAL wal, final long myseqid,
+                final Collection<HStore> storesToFlush, MonitoredTask status,
                 boolean writeFlushWalMarker)
                     throws IOException {
               LOG.info("InternalFlushCache Invoked");
-              FlushResult fs = super.internalFlushcache(wal, myseqid, storesToFlush,
+              FlushResultImpl fs = super.internalFlushcache(wal, myseqid, storesToFlush,
                   Mockito.mock(MonitoredTask.class), writeFlushWalMarker);
               flushcount.incrementAndGet();
               return fs;

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
index 656a0c7..d4d22b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -263,7 +264,7 @@ public class TestCoprocessorScanPolicy {
     public InternalScanner preCompactScannerOpen(
         final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
         List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-        InternalScanner s, CompactionRequest request, long readPoint) throws IOException {
+        InternalScanner s,CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
       Long newTtl = ttls.get(store.getTableName());
       Integer newVersions = versions.get(store.getTableName());
       ScanInfo oldSI = store.getScanInfo();


[43/47] hbase git commit: HBASE-18796 Addendum: Scanner returning null incorrectly during locateRegionInMeta retry, resulting in incorrect TableNotFoundException

Posted by bu...@apache.org.
HBASE-18796 Addendum: Scanner returning null incorrectly during locateRegionInMeta retry, resulting in incorrect TableNotFoundException

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 21a61ad597bbc1d15cf2aa44fa9307b580f76c38
Parents: 11d7495
Author: Abhishek Singh Chouhan <ac...@apache.org>
Authored: Fri Sep 22 20:20:23 2017 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Sep 22 10:20:29 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/ConnectionImplementation.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/21a61ad5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 036f597..860eef3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -805,7 +805,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     s.setReversed(true);
     s.withStartRow(metaKey);
     s.addFamily(HConstants.CATALOG_FAMILY);
-    s.setOneRowLimit();
+
     if (this.useMetaReplicas) {
       s.setConsistency(Consistency.TIMELINE);
     }
@@ -835,6 +835,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       try {
         Result regionInfoRow = null;
         s.resetMvccReadPoint();
+        s.setOneRowLimit();
         try (ReversedClientScanner rcs =
             new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory,
                 rpcControllerFactory, getMetaLookupPool(), metaReplicaCallTimeoutScanInMicroSecond)) {


[35/47] hbase git commit: HBASE-18478 Allow users to remove RegionFinder from LoadBalancer calculations if no locality possible

Posted by bu...@apache.org.
HBASE-18478 Allow users to remove RegionFinder from LoadBalancer calculations if no locality possible

This provides significant cluster start time reduction for FileSystems which do not surface locality (S3).

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 8301734594011e7f94f992e6f8c815580a6b62d8
Parents: bc790fe
Author: Zach York <zy...@amazon.com>
Authored: Thu Jun 22 14:12:49 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Sep 19 17:35:04 2017 -0700

----------------------------------------------------------------------
 .../hbase/master/balancer/BaseLoadBalancer.java | 29 ++++++++++++++++----
 1 file changed, 23 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/83017345/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 162ba92..ca73ff7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -39,6 +39,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
@@ -75,7 +76,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   static final Predicate<ServerLoad> IDLE_SERVER_PREDICATOR
     = load -> load.getNumberOfRegions() == 0;
 
-  protected final RegionLocationFinder regionFinder = new RegionLocationFinder();
+  protected RegionLocationFinder regionFinder;
+  protected boolean useRegionFinder;
 
   private static class DefaultRackManager extends RackManager {
     @Override
@@ -89,6 +91,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
    */
   protected BaseLoadBalancer() {
     metricsBalancer = new MetricsBalancer();
+    createRegionFinder();
   }
 
   /**
@@ -97,6 +100,14 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
    */
   protected BaseLoadBalancer(MetricsBalancer metricsBalancer) {
     this.metricsBalancer = (metricsBalancer != null) ? metricsBalancer : new MetricsBalancer();
+    createRegionFinder();
+  }
+
+  private void createRegionFinder() {
+    useRegionFinder = config.getBoolean("hbase.master.balancer.uselocality", true);
+    if (useRegionFinder) {
+      regionFinder = new RegionLocationFinder();
+    }
   }
 
   /**
@@ -990,7 +1001,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   protected float slop;
   // overallSlop to control simpleLoadBalancer's cluster level threshold
   protected float overallSlop;
-  protected Configuration config;
+  protected Configuration config = HBaseConfiguration.create();
   protected RackManager rackManager;
   private static final Random RANDOM = new Random(System.currentTimeMillis());
   private static final Log LOG = LogFactory.getLog(BaseLoadBalancer.class);
@@ -1019,7 +1030,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       this.tablesOnMaster = true;
     }
     this.rackManager = new RackManager(getConf());
-    regionFinder.setConf(conf);
+    if (useRegionFinder) {
+      regionFinder.setConf(conf);
+    }
     // Print out base configs. Don't print overallSlop since it for simple balancer exclusively.
     LOG.info("slop=" + this.slop + ", tablesOnMaster=" + this.tablesOnMaster +
       ", systemTablesOnMaster=" + this.onlySystemTablesOnMaster);
@@ -1117,7 +1130,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   @Override
   public synchronized void setClusterStatus(ClusterStatus st) {
     this.clusterStatus = st;
-    regionFinder.setClusterStatus(st);
+    if (useRegionFinder) {
+      regionFinder.setClusterStatus(st);
+    }
   }
 
   @Override
@@ -1129,7 +1144,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   public void setMasterServices(MasterServices masterServices) {
     masterServerName = masterServices.getServerName();
     this.services = masterServices;
-    this.regionFinder.setServices(masterServices);
+    if (useRegionFinder) {
+      this.regionFinder.setServices(masterServices);
+    }
   }
 
   public void setRackManager(RackManager rackManager) {
@@ -1271,7 +1288,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
 
   protected Cluster createCluster(List<ServerName> servers,
       Collection<HRegionInfo> regions, boolean forceRefresh) {
-    if (forceRefresh) {
+    if (forceRefresh && useRegionFinder) {
       regionFinder.refreshAndWait(regions);
     }
     // Get the snapshot of the current assignments for the regions in question, and then create


[10/47] hbase git commit: HBASE-17980 Any HRegionInfo we give out should be immutable

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
new file mode 100644
index 0000000..a76767d
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -0,0 +1,624 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.util.Arrays;
+
+@InterfaceAudience.Private
+public class RegionInfoBuilder {
+  private static final Log LOG = LogFactory.getLog(RegionInfoBuilder.class);
+
+  /** A non-capture group so that this can be embedded. */
+  public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
+
+  private static final int MAX_REPLICA_ID = 0xFFFF;
+
+  //TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
+  public static final String NO_HASH = null;
+
+  /**
+   * RegionInfo for first meta region
+   * You cannot use this builder to make an instance of the {@link #FIRST_META_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).
+   */
+  // TODO: How come Meta regions still do not have encoded region names? Fix.
+  // hbase:meta,,1.1588230740 should be the hbase:meta first region name.
+  public static final RegionInfo FIRST_META_REGIONINFO =
+    new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
+
+  private MutableRegionInfo content = null;
+
+  public static RegionInfoBuilder newBuilder(TableName tableName) {
+    return new RegionInfoBuilder(tableName);
+  }
+
+  public static RegionInfoBuilder newBuilder(RegionInfo regionInfo) {
+    return new RegionInfoBuilder(regionInfo);
+  }
+
+  private RegionInfoBuilder(TableName tableName) {
+    this.content = new MutableRegionInfo(tableName);
+  }
+
+  private RegionInfoBuilder(RegionInfo regionInfo) {
+    this.content = new MutableRegionInfo(regionInfo);
+  }
+
+  public RegionInfoBuilder setStartKey(byte[] startKey) {
+    content.setStartKey(startKey);
+    return this;
+  }
+
+  public RegionInfoBuilder setEndKey(byte[] endKey) {
+    content.setEndKey(endKey);
+    return this;
+  }
+
+  public RegionInfoBuilder setRegionId(long regionId) {
+    content.setRegionId(regionId);
+    return this;
+  }
+
+  public RegionInfoBuilder setReplicaId(int replicaId) {
+    content.setReplicaId(replicaId);
+    return this;
+  }
+
+  public RegionInfoBuilder setSplit(boolean isSplit) {
+    content.setSplit(isSplit);
+    return this;
+  }
+
+  public RegionInfoBuilder setOffline(boolean isOffline) {
+    content.setOffline(isOffline);
+    return this;
+  }
+
+  public RegionInfo build() {
+    RegionInfo ri = new MutableRegionInfo(content);
+    // Run a late check that we are not creating default meta region.
+    if (ri.getTable().equals(TableName.META_TABLE_NAME) &&
+        ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
+      throw new IllegalArgumentException("Cannot create the default meta region; " +
+        "use static define FIRST_META_REGIONINFO");
+    }
+    return new MutableRegionInfo(content);
+  }
+
+  /**
+   * An implementation of RegionInfo that adds mutable methods so can build a RegionInfo instance.
+   */
+  @InterfaceAudience.Private
+  static class MutableRegionInfo implements RegionInfo, Comparable<RegionInfo> {
+    /**
+     * The new format for a region name contains its encodedName at the end.
+     * The encoded name also serves as the directory name for the region
+     * in the filesystem.
+     *
+     * New region name format:
+     *    &lt;tablename>,,&lt;startkey>,&lt;regionIdTimestamp>.&lt;encodedName>.
+     * where,
+     *    &lt;encodedName> is a hex version of the MD5 hash of
+     *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
+     *
+     * The old region name format:
+     *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
+     * For region names in the old format, the encoded name is a 32-bit
+     * JenkinsHash integer value (in its decimal notation, string form).
+     *<p>
+     * **NOTE**
+     *
+     * The first hbase:meta region, and regions created by an older
+     * version of HBase (0.20 or prior) will continue to use the
+     * old region name format.
+     */
+
+    // This flag is in the parent of a split while the parent is still referenced
+    // by daughter regions.  We USED to set this flag when we disabled a table
+    // but now table state is kept up in zookeeper as of 0.90.0 HBase.
+    private boolean offLine = false;
+    private boolean split = false;
+    private long regionId = -1;
+    private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
+    private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
+    private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
+    private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
+    private int hashCode = -1;
+    private String encodedName;
+    private byte [] encodedNameAsBytes;
+    // Current TableName
+    private TableName tableName;
+
+    private void setHashCode() {
+      int result = Arrays.hashCode(this.regionName);
+      result ^= this.regionId;
+      result ^= Arrays.hashCode(this.startKey);
+      result ^= Arrays.hashCode(this.endKey);
+      result ^= Boolean.valueOf(this.offLine).hashCode();
+      result ^= Arrays.hashCode(this.tableName.getName());
+      result ^= this.replicaId;
+      this.hashCode = result;
+    }
+
+    /**
+     * Private constructor used constructing MutableRegionInfo for the
+     * first meta regions
+     */
+    private MutableRegionInfo(long regionId, TableName tableName, int replicaId) {
+      // This constructor is currently private for making hbase:meta region only.
+      super();
+      this.regionId = regionId;
+      this.tableName = tableName;
+      this.replicaId = replicaId;
+      // Note: First Meta region replicas names are in old format so we pass false here.
+      this.regionName =
+        RegionInfo.createRegionName(tableName, null, regionId, replicaId, false);
+      setHashCode();
+    }
+
+    MutableRegionInfo(final TableName tableName) {
+      this(tableName, null, null);
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table name
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey)
+    throws IllegalArgumentException {
+      this(tableName, startKey, endKey, false);
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table descriptor
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @param split true if this region has split and we have daughter regions
+     * regions that may or may not hold references to this region.
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
+        final boolean split)
+    throws IllegalArgumentException {
+      this(tableName, startKey, endKey, split, System.currentTimeMillis());
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table descriptor
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @param split true if this region has split and we have daughter regions
+     * regions that may or may not hold references to this region.
+     * @param regionid Region id to use.
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey,
+                       final byte[] endKey, final boolean split, final long regionid)
+    throws IllegalArgumentException {
+      this(tableName, startKey, endKey, split, regionid, RegionInfo.DEFAULT_REPLICA_ID);
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table descriptor
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @param split true if this region has split and we have daughter regions
+     * regions that may or may not hold references to this region.
+     * @param regionid Region id to use.
+     * @param replicaId the replicaId to use
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey,
+                       final byte[] endKey, final boolean split, final long regionid,
+                       final int replicaId)
+      throws IllegalArgumentException {
+      super();
+      if (tableName == null) {
+        throw new IllegalArgumentException("TableName cannot be null");
+      }
+      this.tableName = tableName;
+      this.offLine = false;
+      this.regionId = regionid;
+      this.replicaId = replicaId;
+      if (this.replicaId > MAX_REPLICA_ID) {
+        throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
+      }
+
+      this.regionName = RegionInfo.createRegionName(this.tableName, startKey, regionId, replicaId,
+        !this.tableName.equals(TableName.META_TABLE_NAME));
+
+      this.split = split;
+      this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
+      this.startKey = startKey == null?
+        HConstants.EMPTY_START_ROW: startKey.clone();
+      this.tableName = tableName;
+      setHashCode();
+    }
+
+    /**
+     * Construct MutableRegionInfo.
+     * Only for RegionInfoBuilder to use.
+     * @param other
+     */
+    MutableRegionInfo(MutableRegionInfo other, boolean isMetaRegion) {
+      super();
+      if (other.getTable() == null) {
+        throw new IllegalArgumentException("TableName cannot be null");
+      }
+      this.tableName = other.getTable();
+      this.offLine = other.isOffline();
+      this.regionId = other.getRegionId();
+      this.replicaId = other.getReplicaId();
+      if (this.replicaId > MAX_REPLICA_ID) {
+        throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
+      }
+
+      if(isMetaRegion) {
+        // Note: First Meta region replicas names are in old format
+        this.regionName = RegionInfo.createRegionName(
+                other.getTable(), null, other.getRegionId(),
+                other.getReplicaId(), false);
+      } else {
+        this.regionName = RegionInfo.createRegionName(
+                other.getTable(), other.getStartKey(), other.getRegionId(),
+                other.getReplicaId(), true);
+      }
+
+      this.split = other.isSplit();
+      this.endKey = other.getEndKey() == null? HConstants.EMPTY_END_ROW: other.getEndKey().clone();
+      this.startKey = other.getStartKey() == null?
+        HConstants.EMPTY_START_ROW: other.getStartKey().clone();
+      this.tableName = other.getTable();
+      setHashCode();
+    }
+
+    /**
+     * Construct a copy of RegionInfo as MutableRegionInfo.
+     * Only for RegionInfoBuilder to use.
+     * @param regionInfo
+     */
+    MutableRegionInfo(RegionInfo regionInfo) {
+      super();
+      this.endKey = regionInfo.getEndKey();
+      this.offLine = regionInfo.isOffline();
+      this.regionId = regionInfo.getRegionId();
+      this.regionName = regionInfo.getRegionName();
+      this.split = regionInfo.isSplit();
+      this.startKey = regionInfo.getStartKey();
+      this.hashCode = regionInfo.hashCode();
+      this.encodedName = regionInfo.getEncodedName();
+      this.tableName = regionInfo.getTable();
+      this.replicaId = regionInfo.getReplicaId();
+    }
+
+    /**
+     * @return Return a short, printable name for this region
+     * (usually encoded name) for us logging.
+     */
+    @Override
+    public String getShortNameToLog() {
+      return RegionInfo.prettyPrint(this.getEncodedName());
+    }
+
+    /** @return the regionId */
+    @Override
+    public long getRegionId(){
+      return regionId;
+    }
+
+    /**
+     * set region id.
+     * @param regionId
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setRegionId(long regionId) {
+      this.regionId = regionId;
+      return this;
+    }
+
+    /**
+     * @return the regionName as an array of bytes.
+     * @see #getRegionNameAsString()
+     */
+    @Override
+    public byte [] getRegionName(){
+      return regionName;
+    }
+
+    /**
+     * set region name.
+     * @param regionName
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setRegionName(byte[] regionName) {
+      this.regionName = regionName;
+      return this;
+    }
+
+    /**
+     * @return Region name as a String for use in logging, etc.
+     */
+    @Override
+    public String getRegionNameAsString() {
+      if (RegionInfo.hasEncodedName(this.regionName)) {
+        // new format region names already have their encoded name.
+        return Bytes.toStringBinary(this.regionName);
+      }
+
+      // old format. regionNameStr doesn't have the region name.
+      //
+      //
+      return Bytes.toStringBinary(this.regionName) + "." + this.getEncodedName();
+    }
+
+    /** @return the encoded region name */
+    @Override
+    public synchronized String getEncodedName() {
+      if (this.encodedName == null) {
+        this.encodedName = RegionInfo.encodeRegionName(this.regionName);
+      }
+      return this.encodedName;
+    }
+
+    @Override
+    public synchronized byte [] getEncodedNameAsBytes() {
+      if (this.encodedNameAsBytes == null) {
+        this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
+      }
+      return this.encodedNameAsBytes;
+    }
+
+    /** @return the startKey */
+    @Override
+    public byte [] getStartKey(){
+      return startKey;
+    }
+
+    /**
+     * @param startKey
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setStartKey(byte[] startKey) {
+      this.startKey = startKey;
+      return this;
+    }
+
+    /** @return the endKey */
+    @Override
+    public byte [] getEndKey(){
+      return endKey;
+    }
+
+    /**
+     * @param endKey
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setEndKey(byte[] endKey) {
+      this.endKey = endKey;
+      return this;
+    }
+
+    /**
+     * Get current table name of the region
+     * @return TableName
+     */
+    @Override
+    public TableName getTable() {
+      // This method name should be getTableName but there was already a method getTableName
+      // that returned a byte array.  It is unfortunate given everywhere else, getTableName returns
+      // a TableName instance.
+      if (tableName == null || tableName.getName().length == 0) {
+        tableName = RegionInfo.getTable(getRegionName());
+      }
+      return this.tableName;
+    }
+
+    /**
+     * Returns true if the given inclusive range of rows is fully contained
+     * by this region. For example, if the region is foo,a,g and this is
+     * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
+     * ["b","z"] it will return false.
+     * @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
+     */
+    @Override
+    public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
+      if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
+        throw new IllegalArgumentException(
+        "Invalid range: " + Bytes.toStringBinary(rangeStartKey) +
+        " > " + Bytes.toStringBinary(rangeEndKey));
+      }
+
+      boolean firstKeyInRange = Bytes.compareTo(rangeStartKey, startKey) >= 0;
+      boolean lastKeyInRange =
+        Bytes.compareTo(rangeEndKey, endKey) < 0 ||
+        Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
+      return firstKeyInRange && lastKeyInRange;
+    }
+
+    /**
+     * Return true if the given row falls in this region.
+     */
+    @Override
+    public boolean containsRow(byte[] row) {
+      return Bytes.compareTo(row, startKey) >= 0 &&
+        (Bytes.compareTo(row, endKey) < 0 ||
+         Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
+    }
+
+    /**
+     * @return true if this region is from hbase:meta
+     */
+    @Override
+    public boolean isMetaTable() {
+      return isMetaRegion();
+    }
+
+    /** @return true if this region is a meta region */
+    @Override
+    public boolean isMetaRegion() {
+       return tableName.equals(FIRST_META_REGIONINFO.getTable());
+    }
+
+    /**
+     * @return true if this region is from a system table
+     */
+    @Override
+    public boolean isSystemTable() {
+      return tableName.isSystemTable();
+    }
+
+    /**
+     * @return True if has been split and has daughters.
+     */
+    @Override
+    public boolean isSplit() {
+      return this.split;
+    }
+
+    /**
+     * @param split set split status
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setSplit(boolean split) {
+      this.split = split;
+      return this;
+    }
+
+    /**
+     * @return True if this region is offline.
+     */
+    @Override
+    public boolean isOffline() {
+      return this.offLine;
+    }
+
+    /**
+     * The parent of a region split is offline while split daughters hold
+     * references to the parent. Offlined regions are closed.
+     * @param offLine Set online/offline status.
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setOffline(boolean offLine) {
+      this.offLine = offLine;
+      return this;
+    }
+
+    /**
+     * @return True if this is a split parent region.
+     */
+    @Override
+    public boolean isSplitParent() {
+      if (!isSplit()) return false;
+      if (!isOffline()) {
+        LOG.warn("Region is split but NOT offline: " + getRegionNameAsString());
+      }
+      return true;
+    }
+
+    /**
+     * Returns the region replica id
+     * @return returns region replica id
+     */
+    @Override
+    public int getReplicaId() {
+      return replicaId;
+    }
+
+    public MutableRegionInfo setReplicaId(int replicaId) {
+      this.replicaId = replicaId;
+      return this;
+    }
+
+    /**
+     * @see java.lang.Object#toString()
+     */
+    @Override
+    public String toString() {
+      return "{ENCODED => " + getEncodedName() + ", " +
+        HConstants.NAME + " => '" + Bytes.toStringBinary(this.regionName)
+        + "', STARTKEY => '" +
+        Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
+        Bytes.toStringBinary(this.endKey) + "'" +
+        (isOffline()? ", OFFLINE => true": "") +
+        (isSplit()? ", SPLIT => true": "") +
+        ((replicaId > 0)? ", REPLICA_ID => " + replicaId : "") + "}";
+    }
+
+    /**
+     * @param o
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null) {
+        return false;
+      }
+      if (!(o instanceof RegionInfo)) {
+        return false;
+      }
+      return this.compareTo((RegionInfo)o) == 0;
+    }
+
+    /**
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+      return this.hashCode;
+    }
+
+    @Override
+    public int compareTo(RegionInfo other) {
+      return RegionInfo.COMPARATOR.compare(this, other);
+    }
+
+    /**
+     * @return Comparator to use comparing {@link KeyValue}s.
+     * @deprecated Use Region#getCellComparator().  deprecated for hbase 2.0, remove for hbase 3.0
+     */
+    @Deprecated
+    public KeyValue.KVComparator getComparator() {
+      return isMetaRegion()?
+          KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java
new file mode 100644
index 0000000..7ced1b3
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java
@@ -0,0 +1,135 @@
+/*
+ *
+ * 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.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+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.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+
+/**
+ * Utility used composing RegionInfo for 'display'; e.g. on the web UI
+ */
+@InterfaceAudience.Private
+public class RegionInfoDisplay {
+  public final static String DISPLAY_KEYS_KEY = "hbase.display.keys";
+  public final static byte[] HIDDEN_END_KEY = Bytes.toBytes("hidden-end-key");
+  public final static byte[] HIDDEN_START_KEY = Bytes.toBytes("hidden-start-key");
+
+  /**
+   * Get the descriptive name as {@link RegionState} does it but with hidden
+   * startkey optionally
+   * @return descriptive string
+   */
+  public static String getDescriptiveNameFromRegionStateForDisplay(RegionState state,
+                                                                   Configuration conf) {
+    if (conf.getBoolean(DISPLAY_KEYS_KEY, true)) return state.toDescriptiveString();
+    String descriptiveStringFromState = state.toDescriptiveString();
+    int idx = descriptiveStringFromState.lastIndexOf(" state=");
+    String regionName = getRegionNameAsStringForDisplay(
+    RegionInfoBuilder.newBuilder(state.getRegion()).build(), conf);
+    return regionName + descriptiveStringFromState.substring(idx);
+  }
+
+  /**
+   * Get the end key for display. Optionally hide the real end key.
+   * @return the endkey
+   */
+  public static byte[] getEndKeyForDisplay(RegionInfo ri, Configuration conf) {
+    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
+    if (displayKey) return ri.getEndKey();
+    return HIDDEN_END_KEY;
+  }
+
+  /**
+   * Get the start key for display. Optionally hide the real start key.
+   * @param ri
+   * @param conf
+   * @return the startkey
+   */
+  public static byte[] getStartKeyForDisplay(RegionInfo ri, Configuration conf) {
+    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
+    if (displayKey) return ri.getStartKey();
+    return HIDDEN_START_KEY;
+  }
+
+  /**
+   * Get the region name for display. Optionally hide the start key.
+   * @param ri
+   * @param conf
+   * @return region name as String
+   */
+  public static String getRegionNameAsStringForDisplay(RegionInfo ri, Configuration conf) {
+    return Bytes.toStringBinary(getRegionNameForDisplay(ri, conf));
+  }
+
+  /**
+   * Get the region name for display. Optionally hide the start key.
+   * @param ri
+   * @param conf
+   * @return region name bytes
+   */
+  public static byte[] getRegionNameForDisplay(RegionInfo ri, Configuration conf) {
+    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
+    if (displayKey || ri.getTable().equals(TableName.META_TABLE_NAME)) {
+      return ri.getRegionName();
+    } else {
+      // create a modified regionname with the startkey replaced but preserving
+      // the other parts including the encodedname.
+      try {
+        byte[][]regionNameParts = RegionInfo.parseRegionName(ri.getRegionName());
+        regionNameParts[1] = HIDDEN_START_KEY; //replace the real startkey
+        int len = 0;
+        // get the total length
+        for (byte[] b : regionNameParts) {
+          len += b.length;
+        }
+        byte[] encodedRegionName =
+        Bytes.toBytes(RegionInfo.encodeRegionName(ri.getRegionName()));
+        len += encodedRegionName.length;
+        //allocate some extra bytes for the delimiters and the last '.'
+        byte[] modifiedName = new byte[len + regionNameParts.length + 1];
+        int lengthSoFar = 0;
+        int loopCount = 0;
+        for (byte[] b : regionNameParts) {
+          System.arraycopy(b, 0, modifiedName, lengthSoFar, b.length);
+          lengthSoFar += b.length;
+          if (loopCount++ == 2) modifiedName[lengthSoFar++] = RegionInfo.REPLICA_ID_DELIMITER;
+          else  modifiedName[lengthSoFar++] = HConstants.DELIMITER;
+        }
+        // replace the last comma with '.'
+        modifiedName[lengthSoFar - 1] = RegionInfo.ENC_SEPARATOR;
+        System.arraycopy(encodedRegionName, 0, modifiedName, lengthSoFar,
+        encodedRegionName.length);
+        lengthSoFar += encodedRegionName.length;
+        modifiedName[lengthSoFar] = RegionInfo.ENC_SEPARATOR;
+        return modifiedName;
+      } catch (IOException e) {
+        //LOG.warn("Encountered exception " + e);
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
index 502e0a5..2c1d478 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
@@ -57,6 +57,7 @@ public class RegionReplicaUtil {
    * @return an HRegionInfo object corresponding to the same range (table, start and
    * end key), but for the given replicaId.
    */
+  @Deprecated // Deprecated for HBase-2.0.0, use #getRegionInfoForReplica
   public static HRegionInfo getRegionInfoForReplica(HRegionInfo regionInfo, int replicaId) {
     if (regionInfo.getReplicaId() == replicaId) {
       return regionInfo;
@@ -73,6 +74,37 @@ public class RegionReplicaUtil {
   }
 
   /**
+   * Returns the RegionInfo for the given replicaId.
+   * RegionInfo's correspond to a range of a table, but more than one
+   * "instance" of the same range can be deployed which are differentiated by
+   * the replicaId.
+   * @param regionInfo
+   * @param replicaId the replicaId to use
+   * @return an RegionInfo object corresponding to the same range (table, start and
+   * end key), but for the given replicaId.
+   */
+  public static RegionInfo getRegionInfoForReplica(RegionInfo regionInfo, int replicaId) {
+    if (regionInfo.getReplicaId() == replicaId) {
+      return regionInfo;
+    }
+    RegionInfoBuilder replicaInfo;
+    RegionInfo ri;
+    if (regionInfo.isMetaRegion()) {
+      ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+    } else {
+      replicaInfo = RegionInfoBuilder.newBuilder(regionInfo.getTable())
+              .setStartKey(regionInfo.getStartKey())
+              .setEndKey(regionInfo.getEndKey())
+              .setSplit(regionInfo.isSplit())
+              .setRegionId(regionInfo.getRegionId())
+              .setReplicaId(replicaId);
+      replicaInfo.setOffline(regionInfo.isOffline());
+      ri = replicaInfo.build();
+    }
+    return ri;
+  }
+
+  /**
    * Returns the HRegionInfo for the default replicaId (0). HRegionInfo's correspond to
    * a range of a table, but more than one "instance" of the same range can be
    * deployed which are differentiated by the replicaId.

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
index 89900d4..dfe04e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 class UnmodifyableHRegionInfo extends HRegionInfo {
   /*
    * Creates an unmodifyable copy of an HRegionInfo

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/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 711315f..3c9738e 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
@@ -79,7 +79,9 @@ import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLoadStats;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
@@ -3306,4 +3308,66 @@ public final class ProtobufUtil {
     }
     return lockedResourceJsons.toString();
   }
+
+  /**
+   * Convert a RegionInfo to a Proto RegionInfo
+   *
+   * @param info the RegionInfo to convert
+   * @return the converted Proto RegionInfo
+   */
+  public static HBaseProtos.RegionInfo toProtoRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
+    if (info == null) return null;
+    HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
+    builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
+    builder.setRegionId(info.getRegionId());
+    if (info.getStartKey() != null) {
+      builder.setStartKey(UnsafeByteOperations.unsafeWrap(info.getStartKey()));
+    }
+    if (info.getEndKey() != null) {
+      builder.setEndKey(UnsafeByteOperations.unsafeWrap(info.getEndKey()));
+    }
+    builder.setOffline(info.isOffline());
+    builder.setSplit(info.isSplit());
+    builder.setReplicaId(info.getReplicaId());
+    return builder.build();
+  }
+
+  /**
+   * Convert HBaseProto.RegionInfo to a RegionInfo
+   *
+   * @param proto the RegionInfo to convert
+   * @return the converted RegionInfo
+   */
+  public static org.apache.hadoop.hbase.client.RegionInfo toRegionInfo(final HBaseProtos.RegionInfo proto) {
+    if (proto == null) return null;
+    TableName tableName = ProtobufUtil.toTableName(proto.getTableName());
+    long regionId = proto.getRegionId();
+    int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
+    int replicaId = proto.hasReplicaId()? proto.getReplicaId(): defaultReplicaId;
+    if (tableName.equals(TableName.META_TABLE_NAME) && replicaId == defaultReplicaId) {
+      return RegionInfoBuilder.FIRST_META_REGIONINFO;
+    }
+    byte[] startKey = null;
+    byte[] endKey = null;
+    if (proto.hasStartKey()) {
+      startKey = proto.getStartKey().toByteArray();
+    }
+    if (proto.hasEndKey()) {
+      endKey = proto.getEndKey().toByteArray();
+    }
+    boolean split = false;
+    if (proto.hasSplit()) {
+      split = proto.getSplit();
+    }
+    RegionInfoBuilder rib = RegionInfoBuilder.newBuilder(tableName)
+    .setStartKey(startKey)
+    .setEndKey(endKey)
+    .setRegionId(regionId)
+    .setReplicaId(replicaId)
+    .setSplit(split);
+    if (proto.hasOffline()) {
+      rib.setOffline(proto.getOffline());
+    }
+    return rib.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index 0787f11..dada632 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -443,7 +443,8 @@ public class MetaTableLocator {
       LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
       return;
     }
-    LOG.info("Setting hbase:meta region location in ZooKeeper as " + serverName);
+    LOG.info("Setting hbase:meta (replicaId=" + replicaId + ") location in ZooKeeper as " +
+      serverName);
     // Make the MetaRegionServer pb and then get its bytes and save this as
     // the znode content.
     MetaRegionServer pbrsr = MetaRegionServer.newBuilder()
@@ -458,7 +459,7 @@ public class MetaTableLocator {
       if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
         LOG.debug("META region location doesn't exist, create it");
       } else {
-        LOG.debug("META region location doesn't exist for replicaId " + replicaId +
+        LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
             ", create it");
       }
       ZKUtil.createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data);

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java
new file mode 100644
index 0000000..4644641
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import static org.junit.Assert.fail;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test ImmutableHRegionInfo
+ */
+@Category({ClientTests.class, SmallTests.class})
+public class TestImmutableHRegionInfo {
+
+  @Rule
+  public TestName name = new TestName();
+
+  private final List<Consumer<ImmutableHRegionInfo>> TEST_FUNCTIONS = Arrays.asList(
+    hri -> hri.setOffline(true),
+    hri -> hri.setSplit(true)
+  );
+
+  @Test
+  public void testImmutable() {
+    HRegionInfo hri = new HRegionInfo(TableName.valueOf(name.getMethodName()));
+    ImmutableHRegionInfo immutableHri = new ImmutableHRegionInfo(hri);
+
+    TEST_FUNCTIONS.forEach(f -> {
+      try {
+        f.accept(immutableHri);
+        fail("ImmutableHRegionInfo can't be modified !!!");
+      } catch(UnsupportedOperationException e) {
+      }
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java
new file mode 100644
index 0000000..978e8c8
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.HRegionInfo;
+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.RegionInfoDisplay;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+
+import java.io.IOException;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestRegionInfoDisplay {
+  @Rule
+  public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+  withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
+
+  @Test
+  public void testRegionDetailsForDisplay() throws IOException {
+    byte[] startKey = new byte[] {0x01, 0x01, 0x02, 0x03};
+    byte[] endKey = new byte[] {0x01, 0x01, 0x02, 0x04};
+    Configuration conf = new Configuration();
+    conf.setBoolean("hbase.display.keys", false);
+    RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+      .setStartKey(startKey).setEndKey(endKey).build();
+    checkEquality(ri, conf);
+    // check HRIs with non-default replicaId
+    ri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+    .setStartKey(startKey)
+    .setEndKey(endKey)
+    .setSplit(false)
+    .setRegionId(System.currentTimeMillis())
+    .setReplicaId(1).build();
+    checkEquality(ri, conf);
+    Assert.assertArrayEquals(RegionInfoDisplay.HIDDEN_END_KEY,
+    RegionInfoDisplay.getEndKeyForDisplay(ri, conf));
+    Assert.assertArrayEquals(RegionInfoDisplay.HIDDEN_START_KEY,
+    RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
+
+    RegionState state = new RegionState(convert(ri), RegionState.State.OPEN);
+    String descriptiveNameForDisplay =
+    RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
+    checkDescriptiveNameEquality(descriptiveNameForDisplay,state.toDescriptiveString(), startKey);
+
+    conf.setBoolean("hbase.display.keys", true);
+    Assert.assertArrayEquals(endKey, RegionInfoDisplay.getEndKeyForDisplay(ri, conf));
+    Assert.assertArrayEquals(startKey, RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
+    Assert.assertEquals(state.toDescriptiveString(),
+    RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf));
+  }
+
+  private void checkDescriptiveNameEquality(String descriptiveNameForDisplay, String origDesc,
+                                            byte[] startKey) {
+    // except for the "hidden-start-key" substring everything else should exactly match
+    String firstPart = descriptiveNameForDisplay.substring(0,
+    descriptiveNameForDisplay.indexOf(new String(RegionInfoDisplay.HIDDEN_START_KEY)));
+    String secondPart = descriptiveNameForDisplay.substring(
+    descriptiveNameForDisplay.indexOf(new String(RegionInfoDisplay.HIDDEN_START_KEY)) +
+    RegionInfoDisplay.HIDDEN_START_KEY.length);
+    String firstPartOrig = origDesc.substring(0,
+    origDesc.indexOf(Bytes.toStringBinary(startKey)));
+    String secondPartOrig = origDesc.substring(
+    origDesc.indexOf(Bytes.toStringBinary(startKey)) +
+    Bytes.toStringBinary(startKey).length());
+    assert(firstPart.equals(firstPartOrig));
+    assert(secondPart.equals(secondPartOrig));
+  }
+
+  private void checkEquality(RegionInfo ri, Configuration conf) throws IOException {
+    byte[] modifiedRegionName = RegionInfoDisplay.getRegionNameForDisplay(ri, conf);
+    System.out.println(Bytes.toString(modifiedRegionName) + " " + ri.toString());
+    byte[][] modifiedRegionNameParts = RegionInfo.parseRegionName(modifiedRegionName);
+    byte[][] regionNameParts = RegionInfo.parseRegionName(ri.getRegionName());
+
+    //same number of parts
+    assert(modifiedRegionNameParts.length == regionNameParts.length);
+    for (int i = 0; i < regionNameParts.length; i++) {
+      // all parts should match except for [1] where in the modified one,
+      // we should have "hidden_start_key"
+      if (i != 1) {
+        System.out.println("" + i + " " + Bytes.toString(regionNameParts[i]) + " " +
+          Bytes.toString(modifiedRegionNameParts[i]));
+        Assert.assertArrayEquals(regionNameParts[i], modifiedRegionNameParts[i]);
+      } else {
+        System.out.println("" + i + " " + Bytes.toString(regionNameParts[i]) + " " +
+          Bytes.toString(modifiedRegionNameParts[i]));
+        Assert.assertNotEquals(regionNameParts[i], modifiedRegionNameParts[i]);
+        Assert.assertArrayEquals(modifiedRegionNameParts[1],
+          RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
+      }
+    }
+  }
+
+  private HRegionInfo convert(RegionInfo ri) {
+    HRegionInfo hri = new HRegionInfo(
+    ri.getTable(), ri.getStartKey(), ri.getEndKey(), ri.isSplit(), ri.getRegionId());
+    hri.setOffline(ri.isOffline());
+    return hri;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
index b5e6dd0..bb3686b 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
@@ -17,7 +17,6 @@ See the License for the specific language governing permissions and
 limitations under the License.
 </%doc>
 <%import>
-org.apache.hadoop.hbase.HRegionInfo;
 org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 org.apache.hadoop.hbase.master.assignment.AssignmentManager.RegionInTransitionStat;
 org.apache.hadoop.hbase.master.assignment.RegionStates.RegionFailedOpen;
@@ -25,6 +24,7 @@ org.apache.hadoop.hbase.master.RegionState;
 org.apache.hadoop.conf.Configuration;
 org.apache.hadoop.hbase.HBaseConfiguration;
 org.apache.hadoop.hbase.HConstants;
+org.apache.hadoop.hbase.client.RegionInfoDisplay;
 java.util.HashSet;
 java.util.SortedSet;
 java.util.Map;
@@ -94,7 +94,7 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
                           }
                         </%java>
                         <td><% rs.getRegion().getEncodedName() %></td><td>
-                        <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(rs,
+                        <% RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(rs,
                             assignmentManager.getConfiguration()) %></td>
                         <td><% (currentTime - rs.getStamp()) %> </td>
                         <td> <% retryStatus %> </td>

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/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 f58cbaf..3a9c34a 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
@@ -19,37 +19,36 @@
 
 package org.apache.hadoop.hbase.master.assignment;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 
 /**
  * RegionStates contains a set of Maps that describes the in-memory state of the AM, with

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
index 9a1515b..a89237e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
@@ -105,17 +105,4 @@ public class MetaMockingUtil {
     return Result.create(kvs);
   }
 
-  /**
-   * @param sn  ServerName to use making startcode and server in meta
-   * @param hri Region to serialize into HRegionInfo
-   * @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
-   * @throws IOException
-   */
-  public static Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri,
-      final ServerName sn) throws IOException {
-    hri.setOffline(true);
-    hri.setSplit(true);
-    return getMetaTableRowResult(hri, sn);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
index 003dfdd..0087ecd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
@@ -18,16 +18,6 @@
 
 package org.apache.hadoop.hbase.master.assignment;
 
-import static org.junit.Assert.assertEquals;
-
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -45,6 +35,16 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+
 @Category({MasterTests.class, MediumTests.class})
 public class TestRegionStates {
   private static final Log LOG = LogFactory.getLog(TestRegionStates.class);
@@ -217,8 +217,4 @@ public class TestRegionStates {
         StringUtils.humanTimeDiff(et - st),
       StringUtils.humanSize(NRUNS / ((et - st) / 1000.0f))));
   }
-
-  // ==========================================================================
-  //  Server related
-  // ==========================================================================
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
new file mode 100644
index 0000000..497ecc4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
@@ -0,0 +1,323 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+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.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category({RegionServerTests.class, SmallTests.class})
+public class TestRegionInfoBuilder {
+  @Rule
+  public TestName name = new TestName();
+
+  @Test
+  public void testBuilder() {
+    TableName tn = TableName.valueOf("test");
+    RegionInfoBuilder builder = RegionInfoBuilder.newBuilder(tn);
+    byte[] startKey = Bytes.toBytes("a");
+    builder.setStartKey(startKey);
+    byte[] endKey = Bytes.toBytes("z");
+    builder.setEndKey(endKey);
+    int regionId = 1;
+    builder.setRegionId(1);
+    int replicaId = 2;
+    builder.setReplicaId(replicaId);
+    boolean offline = true;
+    builder.setOffline(offline);
+    boolean isSplit = true;
+    builder.setSplit(isSplit);
+    RegionInfo ri = builder.build();
+
+    assertEquals(tn, ri.getTable());
+    assertArrayEquals(startKey, ri.getStartKey());
+    assertArrayEquals(endKey, ri.getEndKey());
+    assertEquals(regionId, ri.getRegionId());
+    assertEquals(replicaId, ri.getReplicaId());
+    assertEquals(offline, ri.isOffline());
+    assertEquals(isSplit, ri.isSplit());
+  }
+
+  @Test
+  public void testPb() throws DeserializationException {
+    RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+    byte [] bytes = RegionInfo.toByteArray(ri);
+    RegionInfo pbri = RegionInfo.parseFrom(bytes);
+    assertTrue(ri.equals(pbri));
+  }
+
+  @Test
+  public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedException {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+    Path basedir = htu.getDataTestDir();
+    // Create a region.  That'll write the .regioninfo file.
+    FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration());
+    HRegion r = HBaseTestingUtility.createRegionAndWAL(convert(ri), basedir, htu.getConfiguration(),
+        fsTableDescriptors.get(TableName.META_TABLE_NAME));
+    // Get modtime on the file.
+    long modtime = getModTime(r);
+    HBaseTestingUtility.closeRegionAndWAL(r);
+    Thread.sleep(1001);
+    r = HRegion.openHRegion(basedir, convert(ri), fsTableDescriptors.get(TableName.META_TABLE_NAME),
+        null, htu.getConfiguration());
+    // Ensure the file is not written for a second time.
+    long modtime2 = getModTime(r);
+    assertEquals(modtime, modtime2);
+    // Now load the file.
+    RegionInfo deserializedRi = HRegionFileSystem.loadRegionInfoFileContent(
+        r.getRegionFileSystem().getFileSystem(), r.getRegionFileSystem().getRegionDir());
+    HBaseTestingUtility.closeRegionAndWAL(r);
+  }
+
+  long getModTime(final HRegion r) throws IOException {
+    FileStatus[] statuses = r.getRegionFileSystem().getFileSystem().listStatus(
+      new Path(r.getRegionFileSystem().getRegionDir(), HRegionFileSystem.REGION_INFO_FILE));
+    assertTrue(statuses != null && statuses.length == 1);
+    return statuses[0].getModificationTime();
+  }
+
+  @Test
+  public void testCreateRegionInfoName() throws Exception {
+    final String tableName = name.getMethodName();
+    final TableName tn = TableName.valueOf(tableName);
+    String startKey = "startkey";
+    final byte[] sk = Bytes.toBytes(startKey);
+    String id = "id";
+
+    // old format region name
+    byte [] name = RegionInfo.createRegionName(tn, sk, id, false);
+    String nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id, nameStr);
+
+
+    // new format region name.
+    String md5HashInHex = MD5Hash.getMD5AsHex(name);
+    assertEquals(RegionInfo.MD5_HEX_LENGTH, md5HashInHex.length());
+    name = RegionInfo.createRegionName(tn, sk, id, true);
+    nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + ","
+                 + id + "." + md5HashInHex + ".",
+                 nameStr);
+  }
+
+  @Test
+  public void testContainsRange() {
+    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(
+            TableName.valueOf(name.getMethodName())).build();
+    RegionInfo ri = RegionInfoBuilder.newBuilder(tableDesc.getTableName())
+            .setStartKey(Bytes.toBytes("a"))
+            .setEndKey(Bytes.toBytes("g")).build();
+    // Single row range at start of region
+    assertTrue(ri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("a")));
+    // Fully contained range
+    assertTrue(ri.containsRange(Bytes.toBytes("b"), Bytes.toBytes("c")));
+    // Range overlapping start of region
+    assertTrue(ri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("c")));
+    // Fully contained single-row range
+    assertTrue(ri.containsRange(Bytes.toBytes("c"), Bytes.toBytes("c")));
+    // Range that overlaps end key and hence doesn't fit
+    assertFalse(ri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("g")));
+    // Single row range on end key
+    assertFalse(ri.containsRange(Bytes.toBytes("g"), Bytes.toBytes("g")));
+    // Single row range entirely outside
+    assertFalse(ri.containsRange(Bytes.toBytes("z"), Bytes.toBytes("z")));
+
+    // Degenerate range
+    try {
+      ri.containsRange(Bytes.toBytes("z"), Bytes.toBytes("a"));
+      fail("Invalid range did not throw IAE");
+    } catch (IllegalArgumentException iae) {
+    }
+  }
+
+  @Test
+  public void testLastRegionCompare() {
+    TableDescriptor tableDesc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    RegionInfo rip = RegionInfoBuilder.newBuilder(tableDesc.getTableName())
+            .setStartKey(Bytes.toBytes("a"))
+            .setEndKey(new byte[0]).build();
+    RegionInfo ric = RegionInfoBuilder.newBuilder(tableDesc.getTableName())
+            .setStartKey(Bytes.toBytes("a"))
+            .setEndKey(Bytes.toBytes("b")).build();
+    assertTrue(RegionInfo.COMPARATOR.compare(rip, ric) > 0);
+  }
+
+  @Test
+  public void testMetaTables() {
+    assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaTable());
+  }
+
+  @Test
+  public void testComparator() {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    byte[] empty = new byte[0];
+    RegionInfo older = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(empty)
+            .setEndKey(empty)
+            .setSplit(false)
+            .setRegionId(0L).build();
+    RegionInfo newer = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(empty)
+            .setEndKey(empty)
+            .setSplit(false)
+            .setRegionId(1L).build();
+    assertTrue(RegionInfo.COMPARATOR.compare(older, newer) < 0);
+    assertTrue(RegionInfo.COMPARATOR.compare(newer, older) > 0);
+    assertTrue(RegionInfo.COMPARATOR.compare(older, older) == 0);
+    assertTrue(RegionInfo.COMPARATOR.compare(newer, newer) == 0);
+  }
+
+  @Test
+  public void testRegionNameForRegionReplicas() throws Exception {
+    String tableName = name.getMethodName();
+    final TableName tn = TableName.valueOf(tableName);
+    String startKey = "startkey";
+    final byte[] sk = Bytes.toBytes(startKey);
+    String id = "id";
+
+    // assert with only the region name without encoding
+
+    // primary, replicaId = 0
+    byte [] name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0, false);
+    String nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id, nameStr);
+
+    // replicaId = 1
+    name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 1, false);
+    nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id + "_" +
+      String.format(RegionInfo.REPLICA_ID_FORMAT, 1), nameStr);
+
+    // replicaId = max
+    name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0xFFFF, false);
+    nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id + "_" +
+        String.format(RegionInfo.REPLICA_ID_FORMAT, 0xFFFF), nameStr);
+  }
+
+  @Test
+  public void testParseName() throws IOException {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    byte[] startKey = Bytes.toBytes("startKey");
+    long regionId = System.currentTimeMillis();
+    int replicaId = 42;
+
+    // test without replicaId
+    byte[] regionName = RegionInfo.createRegionName(tableName, startKey, regionId, false);
+
+    byte[][] fields = RegionInfo.parseRegionName(regionName);
+    assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
+    assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
+    assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
+    assertEquals(3, fields.length);
+
+    // test with replicaId
+    regionName = RegionInfo.createRegionName(tableName, startKey, regionId,
+      replicaId, false);
+
+    fields = RegionInfo.parseRegionName(regionName);
+    assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
+    assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
+    assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
+    assertArrayEquals(Bytes.toString(fields[3]), Bytes.toBytes(
+      String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)), fields[3]);
+  }
+
+  @Test
+  public void testConvert() {
+    final TableName tableName = TableName.valueOf("ns1:" + name.getMethodName());
+    byte[] startKey = Bytes.toBytes("startKey");
+    byte[] endKey = Bytes.toBytes("endKey");
+    boolean split = false;
+    long regionId = System.currentTimeMillis();
+    int replicaId = 42;
+
+
+    RegionInfo ri = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(startKey)
+            .setEndKey(endKey)
+            .setSplit(split)
+            .setRegionId(regionId)
+            .setReplicaId(replicaId).build();
+
+    // convert two times, compare
+    RegionInfo convertedRi = ProtobufUtil.toRegionInfo(ProtobufUtil.toProtoRegionInfo(ri));
+
+    assertEquals(ri, convertedRi);
+
+    // test convert RegionInfo without replicaId
+    HBaseProtos.RegionInfo info = HBaseProtos.RegionInfo.newBuilder()
+      .setTableName(HBaseProtos.TableName.newBuilder()
+        .setQualifier(UnsafeByteOperations.unsafeWrap(tableName.getQualifier()))
+        .setNamespace(UnsafeByteOperations.unsafeWrap(tableName.getNamespace()))
+        .build())
+      .setStartKey(UnsafeByteOperations.unsafeWrap(startKey))
+      .setEndKey(UnsafeByteOperations.unsafeWrap(endKey))
+      .setSplit(split)
+      .setRegionId(regionId)
+      .build();
+
+    convertedRi = ProtobufUtil.toRegionInfo(info);
+    RegionInfo expectedRi = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(startKey)
+            .setEndKey(endKey)
+            .setSplit(split)
+            .setRegionId(regionId)
+            .setReplicaId(0).build();
+
+    assertEquals(expectedRi, convertedRi);
+  }
+
+  // Duplicated method in TestRegionInfoDisplay too.
+  private HRegionInfo convert(RegionInfo ri) {
+    HRegionInfo hri = new HRegionInfo(
+    ri.getTable(), ri.getStartKey(), ri.getEndKey(), ri.isSplit(), ri.getRegionId());
+    hri.setOffline(ri.isOffline());
+    return hri;
+  }
+}
\ No newline at end of file


[29/47] hbase git commit: HBASE-18832 Fixed LTT, removed references to deprecated HColumnDescriptor

Posted by bu...@apache.org.
HBASE-18832 Fixed LTT, removed references to deprecated HColumnDescriptor

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


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

Branch: refs/heads/HBASE-18467
Commit: 34d648808d262b48bd4986e99a712ceeb8252cfa
Parents: e10d783
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Mon Sep 18 11:12:10 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Sep 18 12:38:02 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/util/LoadTestTool.java  | 40 ++++++++++----------
 1 file changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/34d64880/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
index 0cbcb5f..56dc5c7 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
-import java.util.Random;
 import java.util.concurrent.atomic.AtomicReference;
 
 import javax.crypto.spec.SecretKeySpec;
@@ -36,9 +35,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
@@ -258,46 +258,46 @@ public class LoadTestTool extends AbstractHBaseTool {
       byte[][] columnFamilies) throws IOException {
     try (Connection conn = ConnectionFactory.createConnection(conf);
         Admin admin = conn.getAdmin()) {
-      TableDescriptor tableDesc = admin.getTableDescriptor(tableName);
+      TableDescriptor tableDesc = admin.getDescriptor(tableName);
       LOG.info("Disabling table " + tableName);
       admin.disableTable(tableName);
       for (byte[] cf : columnFamilies) {
-        HColumnDescriptor columnDesc = (HColumnDescriptor) tableDesc.getColumnFamily(cf);
+        ColumnFamilyDescriptor columnDesc = tableDesc.getColumnFamily(cf);
         boolean isNewCf = columnDesc == null;
-        if (isNewCf) {
-          columnDesc = new HColumnDescriptor(cf);
-        }
+        ColumnFamilyDescriptorBuilder columnDescBuilder = isNewCf ?
+            ColumnFamilyDescriptorBuilder.newBuilder(cf) :
+            ColumnFamilyDescriptorBuilder.newBuilder(columnDesc);
         if (bloomType != null) {
-          columnDesc.setBloomFilterType(bloomType);
+          columnDescBuilder.setBloomFilterType(bloomType);
         }
         if (compressAlgo != null) {
-          columnDesc.setCompressionType(compressAlgo);
+          columnDescBuilder.setCompressionType(compressAlgo);
         }
         if (dataBlockEncodingAlgo != null) {
-          columnDesc.setDataBlockEncoding(dataBlockEncodingAlgo);
+          columnDescBuilder.setDataBlockEncoding(dataBlockEncodingAlgo);
         }
         if (inMemoryCF) {
-          columnDesc.setInMemory(inMemoryCF);
+          columnDescBuilder.setInMemory(inMemoryCF);
         }
         if (cipher != null) {
           byte[] keyBytes = new byte[cipher.getKeyLength()];
           new SecureRandom().nextBytes(keyBytes);
-          columnDesc.setEncryptionType(cipher.getName());
-          columnDesc.setEncryptionKey(
+          columnDescBuilder.setEncryptionType(cipher.getName());
+          columnDescBuilder.setEncryptionKey(
               EncryptionUtil.wrapKey(conf,
                   User.getCurrent().getShortName(),
                   new SecretKeySpec(keyBytes,
                       cipher.getName())));
         }
         if (mobThreshold >= 0) {
-          columnDesc.setMobEnabled(true);
-          columnDesc.setMobThreshold(mobThreshold);
+          columnDescBuilder.setMobEnabled(true);
+          columnDescBuilder.setMobThreshold(mobThreshold);
         }
 
         if (isNewCf) {
-          admin.addColumnFamily(tableName, columnDesc);
+          admin.addColumnFamily(tableName, columnDescBuilder.build());
         } else {
-          admin.modifyColumnFamily(tableName, columnDesc);
+          admin.modifyColumnFamily(tableName, columnDescBuilder.build());
         }
       }
       LOG.info("Enabling table " + tableName);
@@ -346,7 +346,7 @@ public class LoadTestTool extends AbstractHBaseTool {
         + "already exists");
 
     addOptWithArg(NUM_TABLES,
-      "A positive integer number. When a number n is speicfied, load test "
+      "A positive integer number. When a number n is specified, load test "
           + "tool  will load n table parallely. -tn parameter value becomes "
           + "table name prefix. Each table name is in format <tn>_1...<tn>_n");
 
@@ -705,7 +705,7 @@ public class LoadTestTool extends AbstractHBaseTool {
       LOG.info("Starting to mutate data...");
       System.out.println("Starting to mutate data...");
       // TODO : currently append and increment operations not tested with tags
-      // Will update this aftet it is done
+      // Will update this after it is done
       updaterThreads.start(startKey, endKey, numUpdaterThreads);
     }
 
@@ -897,7 +897,7 @@ public class LoadTestTool extends AbstractHBaseTool {
 
   private void addAuthInfoToConf(Properties authConfig, Configuration conf, String owner,
       String userList) throws IOException {
-    List<String> users = new ArrayList(Arrays.asList(userList.split(",")));
+    List<String> users = new ArrayList<>(Arrays.asList(userList.split(",")));
     users.add(owner);
     for (String user : users) {
       String keyTabFileConfKey = "hbase." + user + ".keytab.file";


[20/47] hbase git commit: HBASE-18798 Remove the unused methods in RegionServerObserver

Posted by bu...@apache.org.
HBASE-18798 Remove the unused methods in RegionServerObserver


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

Branch: refs/heads/HBASE-18467
Commit: 190a660d17c2a2cca6128edb33254e7ff851bbb3
Parents: 5c12e42
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Fri Sep 15 19:00:43 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Sep 18 09:44:57 2017 +0800

----------------------------------------------------------------------
 .../hbase/coprocessor/RegionServerObserver.java | 62 ------------------
 .../RegionServerCoprocessorHost.java            | 69 --------------------
 .../hbase/security/access/AccessController.java | 27 --------
 .../hbase/namespace/TestNamespaceAuditor.java   | 13 ----
 .../security/access/TestAccessController.java   | 25 -------
 .../access/TestWithDisabledAuthorization.java   | 15 -----
 6 files changed, 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/190a660d/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
index 9fce32a..4e31d19 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
@@ -24,12 +24,9 @@ import java.util.List;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 
 /**
@@ -68,65 +65,6 @@ public interface RegionServerObserver extends Coprocessor {
     final ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {}
 
   /**
-   * Called before the regions merge.
-   * Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} to skip the merge.
-   * @param ctx the environment to interact with the framework and region server.
-   * @param regionA region being merged.
-   * @param regionB region being merged.
-   */
-  default void preMerge(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      final Region regionA, final Region regionB) throws IOException {}
-
-  /**
-   * called after the regions merge.
-   * @param ctx the environment to interact with the framework and region server.
-   * @param regionA region being merged.
-   * @param regionB region being merged.
-   */
-  default void postMerge(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      final Region regionA, final Region regionB, final Region mergedRegion) throws IOException {}
-
-  /**
-   * This will be called before PONR step as part of regions merge transaction. Calling
-   * {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} rollback the merge
-   * @param ctx the environment to interact with the framework and region server.
-   * @param regionA region being merged.
-   * @param regionB region being merged.
-   * @param metaEntries mutations to execute on hbase:meta atomically with regions merge updates.
-   *        Any puts or deletes to execute on hbase:meta can be added to the mutations.
-   */
-  default void preMergeCommit(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      final Region regionA, final Region regionB,
-      @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException {}
-
-  /**
-   * This will be called after PONR step as part of regions merge transaction.
-   * @param ctx the environment to interact with the framework and region server.
-   * @param regionA region being merged.
-   * @param regionB region being merged.
-   */
-  default void postMergeCommit(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      final Region regionA, final Region regionB, final Region mergedRegion) throws IOException {}
-
-  /**
-   * This will be called before the roll back of the regions merge.
-   * @param ctx the environment to interact with the framework and region server.
-   * @param regionA region being merged.
-   * @param regionB region being merged.
-   */
-  default void preRollBackMerge(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      final Region regionA, final Region regionB) throws IOException {}
-
-  /**
-   * This will be called after the roll back of the regions merge.
-   * @param ctx the environment to interact with the framework and region server.
-   * @param regionA region being merged.
-   * @param regionB region being merged.
-   */
-  default void postRollBackMerge(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      final Region regionA, final Region regionB) throws IOException {}
-
-  /**
    * This will be called before executing user request to roll a region server WAL.
    * @param ctx the environment to interact with the framework and region server.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/190a660d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
index ec58d08..563e4a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java
@@ -30,10 +30,8 @@ import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -97,73 +95,6 @@ public class RegionServerCoprocessorHost extends
     });
   }
 
-  public boolean preMerge(final HRegion regionA, final HRegion regionB, final User user) throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
-      @Override
-      public void call(RegionServerObserver oserver,
-          ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
-        oserver.preMerge(ctx, regionA, regionB);
-      }
-    });
-  }
-
-  public void postMerge(final HRegion regionA, final HRegion regionB, final HRegion mergedRegion,
-                        final User user)
-      throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
-      @Override
-      public void call(RegionServerObserver oserver,
-          ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
-        oserver.postMerge(ctx, regionA, regionB, mergedRegion);
-      }
-    });
-  }
-
-  public boolean preMergeCommit(final HRegion regionA, final HRegion regionB,
-      final @MetaMutationAnnotation List<Mutation> metaEntries, final User user)
-      throws IOException {
-    return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
-      @Override
-      public void call(RegionServerObserver oserver,
-          ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
-        oserver.preMergeCommit(ctx, regionA, regionB, metaEntries);
-      }
-    });
-  }
-
-  public void postMergeCommit(final HRegion regionA, final HRegion regionB,
-      final HRegion mergedRegion, final User user) throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
-      @Override
-      public void call(RegionServerObserver oserver,
-          ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
-        oserver.postMergeCommit(ctx, regionA, regionB, mergedRegion);
-      }
-    });
-  }
-
-  public void preRollBackMerge(final HRegion regionA, final HRegion regionB, final User user)
-      throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
-      @Override
-      public void call(RegionServerObserver oserver,
-          ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
-        oserver.preRollBackMerge(ctx, regionA, regionB);
-      }
-    });
-  }
-
-  public void postRollBackMerge(final HRegion regionA, final HRegion regionB, final User user)
-      throws IOException {
-    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
-      @Override
-      public void call(RegionServerObserver oserver,
-          ObserverContext<RegionServerCoprocessorEnvironment> ctx) throws IOException {
-        oserver.postRollBackMerge(ctx, regionA, regionB);
-      }
-    });
-  }
-
   public void preRollWALWriterRequest() throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/190a660d/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 c34fc6d..4c92fb1 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
@@ -2575,33 +2575,6 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   }
 
   @Override
-  public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
-      Region regionB) throws IOException {
-    requirePermission(getActiveUser(ctx), "mergeRegions", regionA.getTableDescriptor().getTableName(),
-        null, null, Action.ADMIN);
-  }
-
-  @Override
-  public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, Region regionA,
-      Region regionB, Region mergedRegion) throws IOException { }
-
-  @Override
-  public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException { }
-
-  @Override
-  public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      Region regionA, Region regionB, Region mergedRegion) throws IOException { }
-
-  @Override
-  public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      Region regionA, Region regionB) throws IOException { }
-
-  @Override
-  public void postRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-      Region regionA, Region regionB) throws IOException { }
-
-  @Override
   public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
       throws IOException {
     requirePermission(getActiveUser(ctx), "preRollLogWriterRequest", Permission.Action.ADMIN);

http://git-wip-us.apache.org/repos/asf/hbase/blob/190a660d/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 8015115..979936f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -71,11 +70,9 @@ import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.QuotaExceededException;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -294,16 +291,6 @@ public class TestNamespaceAuditor {
         wait();
       }
     }
-
-    @Override
-    public synchronized void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-        Region regionA, Region regionB) throws IOException {
-      triggered = true;
-      notifyAll();
-      if (shouldFailMerge) {
-        throw new IOException("fail merge");
-      }
-    }
   }
 
   public static class CPMasterObserver implements MasterObserver {

http://git-wip-us.apache.org/repos/asf/hbase/blob/190a660d/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index bf36821..2c27e4d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -857,31 +857,6 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test (timeout=180000)
-  public void testMergeRegions() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    createTestTable(tableName);
-    try {
-      final List<HRegion> regions = TEST_UTIL.getHBaseCluster().findRegionsForTable(tableName);
-      assertTrue("not enough regions: " + regions.size(), regions.size() >= 2);
-
-      AccessTestAction action = new AccessTestAction() {
-        @Override
-        public Object run() throws Exception {
-          ACCESS_CONTROLLER.preMerge(ObserverContext.createAndPrepare(RSCP_ENV, null),
-            regions.get(0), regions.get(1));
-          return null;
-        }
-      };
-
-      verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
-      verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
-        USER_GROUP_WRITE, USER_GROUP_CREATE);
-    } finally {
-      deleteTable(TEST_UTIL, tableName);
-    }
-  }
-
-  @Test (timeout=180000)
   public void testFlush() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/190a660d/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
index b118cb5..aab7ae4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
@@ -836,21 +836,6 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
       }
     }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
 
-    // preMerge
-    verifyAllowed(new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
-        Region region_a = mock(Region.class);
-        when(region_a.getTableDescriptor()).thenReturn(htd);
-        Region region_b = mock(Region.class);
-        when(region_b.getTableDescriptor()).thenReturn(htd);
-        ACCESS_CONTROLLER.preMerge(ObserverContext.createAndPrepare(RSCP_ENV, null), region_a,
-          region_b);
-        return null;
-      }
-    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
-
     // preRollWALWriterRequest
     verifyAllowed(new AccessTestAction() {
       @Override


[45/47] hbase git commit: HBASE-18786 FileNotFoundException should not be silently handled for primary region replicas

Posted by bu...@apache.org.
HBASE-18786 FileNotFoundException should not be silently handled for primary region replicas


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

Branch: refs/heads/HBASE-18467
Commit: b27f9b582a858fba66036413936debad27737c3a
Parents: bb9f01c
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Sep 21 13:49:00 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Sep 22 12:45:05 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 106 ++++++-------------
 .../hbase/regionserver/RegionUnassigner.java    |  67 ------------
 2 files changed, 32 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b27f9b58/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 86a24ad..9554d7f 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
@@ -204,9 +204,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
     "hbase.hregion.scan.loadColumnFamiliesOnDemand";
 
-  public static final String HREGION_UNASSIGN_FOR_FNFE = "hbase.hregion.unassign.for.fnfe";
-  public static final boolean DEFAULT_HREGION_UNASSIGN_FOR_FNFE = true;
-
   public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
   public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
 
@@ -654,8 +651,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private final NavigableMap<byte[], Integer> replicationScope = new TreeMap<>(
       Bytes.BYTES_COMPARATOR);
 
-  // whether to unassign region if we hit FNFE
-  private final RegionUnassigner regionUnassigner;
   /**
    * HRegion constructor. This constructor should only be used for testing and
    * extensions.  Instances of HRegion should be instantiated with the
@@ -815,14 +810,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
 
     this.maxCellSize = conf.getLong(HBASE_MAX_CELL_SIZE_KEY, DEFAULT_MAX_CELL_SIZE);
-
-    boolean unassignForFNFE =
-        conf.getBoolean(HREGION_UNASSIGN_FOR_FNFE, DEFAULT_HREGION_UNASSIGN_FOR_FNFE);
-    if (unassignForFNFE) {
-      this.regionUnassigner = new RegionUnassigner(rsServices, fs.getRegionInfo());
-    } else {
-      this.regionUnassigner = null;
-    }
   }
 
   void setHTableSpecificConf() {
@@ -5873,12 +5860,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       try {
         for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
           HStore store = stores.get(entry.getKey());
-          KeyValueScanner scanner;
-          try {
-            scanner = store.getScanner(scan, entry.getValue(), this.readPt);
-          } catch (FileNotFoundException e) {
-            throw handleFileNotFound(e);
-          }
+          KeyValueScanner scanner = store.getScanner(scan, entry.getValue(), this.readPt);
           instantiatedScanners.add(scanner);
           if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
               || this.filter.isFamilyEssential(entry.getKey())) {
@@ -5902,21 +5884,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
     }
 
-    private FileNotFoundException handleFileNotFound(FileNotFoundException fnfe) {
-      // Try reopening the region since we have lost some storefiles.
-      // See HBASE-17712 for more details.
-      LOG.warn("Store file is lost; close and reopen region", fnfe);
-      if (regionUnassigner != null) {
-        regionUnassigner.unassign();
-      }
-      return fnfe;
-    }
-
     private IOException handleException(List<KeyValueScanner> instantiatedScanners,
         Throwable t) {
-      if (t instanceof FileNotFoundException) {
-        handleFileNotFound((FileNotFoundException)t);
-      }
       // remove scaner read point before throw the exception
       scannerReadPoints.remove(this);
       if (storeHeap != null) {
@@ -5999,19 +5968,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         throw new UnknownScannerException("Scanner was closed");
       }
       boolean moreValues = false;
-      try {
-        if (outResults.isEmpty()) {
-          // Usually outResults is empty. This is true when next is called
-          // to handle scan or get operation.
-          moreValues = nextInternal(outResults, scannerContext);
-        } else {
-          List<Cell> tmpList = new ArrayList<Cell>();
-          moreValues = nextInternal(tmpList, scannerContext);
-          outResults.addAll(tmpList);
-        }
-      } catch (FileNotFoundException e) {
-        handleFileNotFound(e);
-        throw e;
+      if (outResults.isEmpty()) {
+        // Usually outResults is empty. This is true when next is called
+        // to handle scan or get operation.
+        moreValues = nextInternal(outResults, scannerContext);
+      } else {
+        List<Cell> tmpList = new ArrayList<Cell>();
+        moreValues = nextInternal(tmpList, scannerContext);
+        outResults.addAll(tmpList);
       }
 
       // If the size limit was reached it means a partial Result is being returned. Returning a
@@ -6061,33 +6025,29 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       boolean tmpKeepProgress = scannerContext.getKeepProgress();
       // Scanning between column families and thus the scope is between cells
       LimitScope limitScope = LimitScope.BETWEEN_CELLS;
-      try {
-        do {
-          // We want to maintain any progress that is made towards the limits while scanning across
-          // different column families. To do this, we toggle the keep progress flag on during calls
-          // to the StoreScanner to ensure that any progress made thus far is not wiped away.
-          scannerContext.setKeepProgress(true);
-          heap.next(results, scannerContext);
-          scannerContext.setKeepProgress(tmpKeepProgress);
-
-          nextKv = heap.peek();
-          moreCellsInRow = moreCellsInRow(nextKv, currentRowCell);
-          if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
-          if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
-            return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
-          } else if (scannerContext.checkSizeLimit(limitScope)) {
-            ScannerContext.NextState state =
-                moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;
-            return scannerContext.setScannerState(state).hasMoreValues();
-          } else if (scannerContext.checkTimeLimit(limitScope)) {
-            ScannerContext.NextState state =
-                moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED;
-            return scannerContext.setScannerState(state).hasMoreValues();
-          }
-        } while (moreCellsInRow);
-      } catch (FileNotFoundException e) {
-        throw handleFileNotFound(e);
-      }
+      do {
+        // We want to maintain any progress that is made towards the limits while scanning across
+        // different column families. To do this, we toggle the keep progress flag on during calls
+        // to the StoreScanner to ensure that any progress made thus far is not wiped away.
+        scannerContext.setKeepProgress(true);
+        heap.next(results, scannerContext);
+        scannerContext.setKeepProgress(tmpKeepProgress);
+
+        nextKv = heap.peek();
+        moreCellsInRow = moreCellsInRow(nextKv, currentRowCell);
+        if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
+        if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
+          return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
+        } else if (scannerContext.checkSizeLimit(limitScope)) {
+          ScannerContext.NextState state =
+              moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;
+          return scannerContext.setScannerState(state).hasMoreValues();
+        } else if (scannerContext.checkTimeLimit(limitScope)) {
+          ScannerContext.NextState state =
+              moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED;
+          return scannerContext.setScannerState(state).hasMoreValues();
+        }
+      } while (moreCellsInRow);
       return nextKv != null;
     }
 
@@ -6435,8 +6395,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (this.joinedHeap != null) {
           result = this.joinedHeap.requestSeek(kv, true, true) || result;
         }
-      } catch (FileNotFoundException e) {
-        throw handleFileNotFound(e);
       } finally {
         closeRegionOperation();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b27f9b58/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java
deleted file mode 100644
index d7eefd1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Used to unssign a region when we hit FNFE.
- */
-@InterfaceAudience.Private
-class RegionUnassigner {
-
-  private static final Log LOG = LogFactory.getLog(RegionUnassigner.class);
-
-  private final RegionServerServices rsServices;
-
-  private final HRegionInfo regionInfo;
-
-  private boolean unassigning = false;
-
-  RegionUnassigner(RegionServerServices rsServices, HRegionInfo regionInfo) {
-    this.rsServices = rsServices;
-    this.regionInfo = regionInfo;
-  }
-
-  synchronized void unassign() {
-    if (unassigning) {
-      return;
-    }
-    unassigning = true;
-    new Thread("RegionUnassigner." + regionInfo.getEncodedName()) {
-      @Override
-      public void run() {
-        LOG.info("Unassign " + regionInfo.getRegionNameAsString());
-        try {
-          rsServices.unassign(regionInfo.getRegionName());
-        } catch (IOException e) {
-          LOG.warn("Unassigned " + regionInfo.getRegionNameAsString() + " failed", e);
-        } finally {
-          synchronized (RegionUnassigner.this) {
-            unassigning = false;
-          }
-        }
-      }
-    }.start();
-  }
-}


[21/47] hbase git commit: HBASE-18836 Note need for explicit javax.el and exclude from shaded artifacts

Posted by bu...@apache.org.
HBASE-18836 Note need for explicit javax.el and exclude from shaded artifacts


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

Branch: refs/heads/HBASE-18467
Commit: de32b11fe4536d07ac4ffee7c59152cebe4f150e
Parents: 190a660
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Sep 18 08:47:22 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Sep 18 08:47:46 2017 -0500

----------------------------------------------------------------------
 hbase-rest/pom.xml                          | 3 +++
 hbase-server/pom.xml                        | 3 +++
 hbase-shaded/hbase-shaded-mapreduce/pom.xml | 4 ++++
 hbase-thrift/pom.xml                        | 3 +++
 4 files changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/de32b11f/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 4553293..0b18809 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -308,6 +308,9 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <!-- Specifically needed for jetty-jsp, included
+         to bypass version scanning that hits a bad repo
+         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/de32b11f/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 03891a1..ad080f1 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -495,6 +495,9 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <!-- Specifically needed for jetty-jsp, included
+         to bypass version scanning that hits a bad repo
+         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/de32b11f/hbase-shaded/hbase-shaded-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-mapreduce/pom.xml b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
index d42735a..2afa3c2 100644
--- a/hbase-shaded/hbase-shaded-mapreduce/pom.xml
+++ b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
@@ -130,6 +130,10 @@
                 <artifactId>jetty-jsp</artifactId>
               </exclusion>
               <exclusion>
+                <groupId>org.glassfish</groupId>
+                <artifactId>javax.el</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>org.eclipse.jetty</groupId>
                 <artifactId>jetty-webapp</artifactId>
               </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/de32b11f/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index 36c992d..0dc0dde 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -299,6 +299,9 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <!-- Specifically needed for jetty-jsp, included
+         to bypass version scanning that hits a bad repo
+         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>


[36/47] hbase git commit: HBASE-18853 hbase-protocol-shaded includes protobuf (since we moved to hbase-thirdparty)

Posted by bu...@apache.org.
HBASE-18853 hbase-protocol-shaded includes protobuf (since we moved to hbase-thirdparty)


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

Branch: refs/heads/HBASE-18467
Commit: e3896cfcc37bc157c75200592e2eaa6be7107856
Parents: 8301734
Author: Michael Stack <st...@apache.org>
Authored: Tue Sep 19 20:39:36 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Sep 20 08:15:01 2017 -0700

----------------------------------------------------------------------
 hbase-protocol-shaded/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e3896cfc/hbase-protocol-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml
index 9bc79c8..fa66a83 100644
--- a/hbase-protocol-shaded/pom.xml
+++ b/hbase-protocol-shaded/pom.xml
@@ -150,7 +150,7 @@
                 <excludes>
                   <!--Exclude protobuf itself. We get a patched version from hbase-thirdparty.
                     -->
-                  <exclude>org.apache.hadoop.hbase.shaded.com.google:*</exclude>
+                  <exclude>org.apache.hbase.thirdparty:*</exclude>
                   <exclude>com.google.protobuf:protobuf-java</exclude>
                   <exclude>com.google.code.findbugs:*</exclude>
                   <exclude>com.google.errorprone:error_prone_annotations</exclude>


[12/47] hbase git commit: HBASE-18766 Make TableSnapshotScanner Audience Private

Posted by bu...@apache.org.
HBASE-18766 Make TableSnapshotScanner Audience Private

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


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

Branch: refs/heads/HBASE-18467
Commit: 9c5b03acd72713f1a9956086ad0a6f4b389deaae
Parents: 58988cb
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Sep 14 06:12:22 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 14 14:30:05 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/9c5b03ac/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
index 849975d..9244ced 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
@@ -62,7 +62,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
  * snapshot/data files will completely circumvent the access control enforced by HBase.
  * See org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat.
  */
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 public class TableSnapshotScanner extends AbstractClientScanner {
 
   private static final Log LOG = LogFactory.getLog(TableSnapshotScanner.class);


[47/47] hbase git commit: HBASE-18467 report nightly results to devs.

Posted by bu...@apache.org.
HBASE-18467 report nightly results to devs.

* run individual stages and track pass/fail instead of fast-fail
* build aggregate results
* post said results to jiras included in git commit messages


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

Branch: refs/heads/HBASE-18467
Commit: 33844651521abcabf65cb75499fd8a0e6f30c8bf
Parents: b483046
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Aug 9 00:48:46 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Sat Sep 23 19:31:20 2017 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile | 132 +++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 126 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/33844651/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 1f01a47..a87d327 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -128,7 +128,18 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       steps {
         unstash 'yetus'
         // TODO should this be a download from master, similar to how the personality is?
-        sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+        sh '''#!/usr/bin/env bash
+          rm -f "${OUTPUTDIR}/success" "${OUTPUTDIR}/failure"
+          declare commentfile
+          if "${BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
+            commentfile="${OUTPUTDIR}/success"
+            echo '(/) *{color:green}+1 general checks{color}*' >> "${commentfile}"
+          else
+            commentfile="${OUTPUTDIR}/failure"
+            echo '(x) *{color:red}-1 general checks{color}*' >> "${commentfile}"
+          fi
+          echo "-- For more information [see general report|${BUILD_URL}/General_Nightly_Build_Report/]" >> "${commentfile}"
+        '''
       }
       post {
         always {
@@ -159,13 +170,22 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       }
       steps {
         unstash 'yetus'
-        sh """#!/usr/bin/env bash
+        sh '''#!/usr/bin/env bash
           # for branch-1.1 we don't do jdk8 findbugs, so do it here
-          if [ "${env.BRANCH_NAME}" == "branch-1.1" ]; then
+          if [ "${BRANCH_NAME}" == "branch-1.1" ]; then
             TESTS+=",findbugs"
           fi
-          "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
-        """
+          declare commentfile
+          rm -f "${OUTPUTDIR}/success" "${OUTPUTDIR}/failure"
+          if "${BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
+            commentfile="${OUTPUTDIR}/success"
+            echo '(/) *{color:green}+1 jdk7 checks{color}*' >> "${commentfile}"
+          else
+            commentfile="${OUTPUTDIR}/failure"
+            echo '(x) *{color:red}-1 jdk7 checks{color}*' >> "${commentfile}"
+          fi
+          echo "-- For more information [see jdk7 report|${BUILD_URL}/JDK7_Nightly_Build_Report/]" >> "${commentfile}"
+        '''
       }
       post {
         always {
@@ -215,7 +235,18 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       }
       steps {
         unstash 'yetus'
-        sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+        sh '''#!/usr/bin/env bash
+          declare commentfile
+          rm -f "${OUTPUTDIR}/success" "${OUTPUTDIR}/failure"
+          if "${BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
+            commentfile="${OUTPUTDIR}/success"
+            echo '(/) *{color:green}+1 jdk8 checks{color}*' >> "${commentfile}"
+          else
+            commentfile="${OUTPUTDIR}/failure"
+            echo '(x) *{color:red}-1 jdk8 checks{color}*' >> "${commentfile}"
+          fi
+          echo "-- For more information [see jdk8 report|${BUILD_URL}/JDK8_Nightly_Build_Report/]" >> "${commentfile}"
+        '''
       }
       post {
         always {
@@ -287,6 +318,7 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
         }
         // expectation check largely based on HBASE-14952
         sh '''#!/bin/bash -e
+          rm -rf "${env.WORKSPACE}/src_tarball_success" "${env.WORKSPACE}/src_tarball_failure"
           echo "Checking against things we don't expect to include in the source tarball (git related, hbase-native-client, etc.)"
           cat >known_excluded <<END
 Only in .: .git
@@ -304,6 +336,94 @@ END
           fi
 '''
       }
+      // This approach only works because the source release artifact is the last stage that does work.
+      post {
+        success {
+          writeFile file: "${env.WORKSPACE}/src_tarball_success", text: '(/) *{color:green}+1 source release artifact{color}*'
+        }
+        failure {
+          writeFile file: "${env.WORKSPACE}/src_tarball_failure", text: '(x) *{color:red}-1 source release artifact{color}*'
+        }
+      }
+    }
+    stage ('Fail if previous stages failed') {
+      steps {
+        script {
+          def failures = ['src_tarball_failure', "${env.OUTPUT_RELATIVE_GENERAL}/failure",
+                          "${env.OUTPUT_RELATIVE_JDK7}/failure", "${OUTPUT_RELATIVE_JDK8}/failure"]
+          for ( failure_file in failures ) {
+            if (fileExists(file: failure_file)) {
+              error 'Failing job due to previous failure(s) in prior steps.'
+            }
+          }
+        }
+      }
+    }
+  }
+  post {
+    always {
+      script {
+         try {
+           sh "printenv"
+           def results = ["${env.OUTPUT_RELATIVE_GENERAL}/failure", "${env.OUTPUT_RELATIVE_GENERAL}/success",
+                          "${env.OUTPUT_RELATIVE_JDK7}/failure", "${env.OUTPUT_RELATIVE_JDK7}/success",
+                          "${env.OUTPUT_RELATIVE_JDK8}/failure", "${env.OUTPUT_RELATIVE_JDK8}/success",
+                          'src_tarball_failure', 'src_tarball_success']
+           echo env.BRANCH_NAME
+           echo env.BUILD_URL
+           echo currentBuild.result
+           echo currentBuild.durationString
+           def comment = "Results for branch ${env.BRANCH_NAME}, done in ${currentBuild.durationString}\n"
+           if (currentBuild.result == "SUCCESS") {
+              comment += '	(/) *{color:green}+1 overall{color}*\\'
+           } else {
+              comment += '(x) *{color:red}-1 overall{color}*\\'
+              // Ideally get the committer our of the change and @ mention them in the per-jira comment
+              comment += 'Committer, please check your recent inclusion of a patch for this issue.\\'
+           }
+           comment += "	[build ${currentBuild.displayName} on builds.a.o|${env.BUILD_URL}]: ${currentBuild.result}\\----\\details (if available):\n"
+           //echo "[DEBUG] Comment so far: "
+           //echo comment
+           echo ""
+           echo "[DEBUG] trying to aggregate step-wise results"
+           comment += results.collect { fileExists(file: it) ? readFile(file: it) : "" }.join("\n")
+           echo "[INFO] Comment:"
+           echo comment
+           echo ""
+           echo "[INFO] There are ${currentBuild.changeSets.size()} change sets."
+           getJirasToComment(currentBuild).each { currentIssue ->
+             jiraComment issueKey: currentIssue, body: comment
+           }
+        } catch (Exception exception) {
+          echo "Got exception: ${exception}"
+          echo "	${exception.getStackTrace()}"
+        }
+      }
+    }
+  }
+}
+import org.jenkinsci.plugins.workflow.support.steps.build.RunWrapper
+@NonCPS
+List<String> getJirasToComment(RunWrapper thisBuild) {
+  def seenJiras = []
+  thisBuild.changeSets.each { cs ->
+    cs.getItems().each { change ->
+      CharSequence msg = change.msg
+      echo "change: ${change}"
+      echo "     ${msg}"
+      echo "     ${change.commitId}"
+      echo "     ${change.author}"
+      echo ""
+      msg.eachMatch("HBASE-[0-9]+") { currentIssue ->
+        echo "[DEBUG] found jira key: ${currentIssue}"
+        if (currentIssue in seenJiras) {
+          echo "[DEBUG] already commented on ${currentIssue}."
+        } else {
+          echo "[INFO] commenting on ${currentIssue}."
+          seenJiras << currentIssue
+        }
+      }
     }
   }
+  return seenJiras
 }


[09/47] hbase git commit: HBASE-18820 assembly is missing hbase-permission

Posted by bu...@apache.org.
HBASE-18820 assembly is missing hbase-permission


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

Branch: refs/heads/HBASE-18467
Commit: a4277f37e281ed3ed8378a525856e18391d2098c
Parents: 98f4171
Author: Michael Stack <st...@apache.org>
Authored: Thu Sep 14 13:25:47 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 14 13:27:47 2017 -0700

----------------------------------------------------------------------
 hbase-assembly/src/main/assembly/src.xml | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a4277f37/hbase-assembly/src/main/assembly/src.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/src.xml b/hbase-assembly/src/main/assembly/src.xml
index a0b700c..3b02b3c 100644
--- a/hbase-assembly/src/main/assembly/src.xml
+++ b/hbase-assembly/src/main/assembly/src.xml
@@ -36,6 +36,9 @@
         <include>org.apache.hbase:hbase-build-support</include>
         <include>org.apache.hbase:hbase-build-configuration</include>
         <include>org.apache.hbase:hbase-assembly</include>
+        <include>org.apache.hbase:hbase-backup</include>
+        <include>org.apache.hbase:hbase-build-configuration</include>
+        <include>org.apache.hbase:hbase-build-support</include>
         <include>org.apache.hbase:hbase-checkstyle</include>
         <include>org.apache.hbase:hbase-client</include>
         <include>org.apache.hbase:hbase-common</include>
@@ -45,12 +48,15 @@
         <include>org.apache.hbase:hbase-hadoop2-compat</include>
         <include>org.apache.hbase:hbase-hadoop-compat</include>
         <include>org.apache.hbase:hbase-it</include>
+        <include>org.apache.hbase:hbase-mapreduce</include>
         <include>org.apache.hbase:hbase-metrics</include>
         <include>org.apache.hbase:hbase-metrics-api</include>
+        <include>org.apache.hbase:hbase-native-client</include>
         <include>org.apache.hbase:hbase-prefix-tree</include>
         <include>org.apache.hbase:hbase-procedure</include>
         <include>org.apache.hbase:hbase-protocol-shaded</include>
         <include>org.apache.hbase:hbase-protocol</include>
+        <include>org.apache.hbase:hbase-replication</include>
         <include>org.apache.hbase:hbase-rest</include>
         <include>org.apache.hbase:hbase-resource-bundle</include>
         <include>org.apache.hbase:hbase-rsgroup</include>
@@ -61,8 +67,6 @@
         <include>org.apache.hbase:hbase-spark-it</include>
         <include>org.apache.hbase:hbase-testing-util</include>
         <include>org.apache.hbase:hbase-thrift</include>
-        <include>org.apache.hbase:hbase-backup</include>
-        <include>org.apache.hbase:hbase-mapreduce</include>
       </includes>
       <!-- Include all the sources in the top directory -->
       <sources>


[37/47] hbase git commit: Revert "HBASE-16478 Rename WALKey in PB to WALEdit This is a rebase of Enis's original patch" Not worth the difference it introduces; means hbase-protocol can no longer parse a WAL entry.

Posted by bu...@apache.org.
Revert "HBASE-16478 Rename WALKey in PB to WALEdit This is a rebase of Enis's original patch"
Not worth the difference it introduces; means hbase-protocol can no
longer parse a WAL entry.

This reverts commit 9a2e680caeb8c6627357ff5a0963170f09e65414.


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

Branch: refs/heads/HBASE-18467
Commit: 37696fffe95bd1cf3c4ca92a4a11560d53df769b
Parents: e3896cf
Author: Michael Stack <st...@apache.org>
Authored: Wed Sep 20 15:27:37 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Sep 20 15:27:37 2017 -0700

----------------------------------------------------------------------
 .../src/main/protobuf/Admin.proto               |  2 +-
 .../src/main/protobuf/WAL.proto                 | 13 +++++--
 .../hbase/protobuf/ReplicationProtbufUtil.java  |  2 +-
 .../hbase/regionserver/RSRpcServices.java       | 30 +++++++++-------
 .../regionserver/wal/ProtobufLogReader.java     |  9 +++--
 .../regionserver/ReplicationSink.java           |  8 ++---
 .../org/apache/hadoop/hbase/wal/WALKey.java     | 36 ++++++++++----------
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 16 ++++-----
 .../regionserver/TestReplicationSink.java       |  5 +--
 9 files changed, 67 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index db5a3be..62aac9a 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -172,7 +172,7 @@ message UpdateFavoredNodesResponse {
 
 // Protocol buffer version of WAL for replication
 message WALEntry {
-  required WALEdit edit = 1;
+  required WALKey key = 1;
   // Following may be null if the KVs/Cells are carried along the side in a cellblock (See
   // RPC for more on cellblocks). If Cells/KVs are in a cellblock, this next field is null
   // and associated_cell_count has count of Cells associated w/ this WALEntry

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-protocol-shaded/src/main/protobuf/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/WAL.proto b/hbase-protocol-shaded/src/main/protobuf/WAL.proto
index 4c9a171..81e5650 100644
--- a/hbase-protocol-shaded/src/main/protobuf/WAL.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/WAL.proto
@@ -34,9 +34,10 @@ message WALHeader {
 }
 
 /*
- * Protocol buffer version of WALEdit;
+ * Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
+ * for some KVs
  */
-message WALEdit {
+message WALKey {
   required bytes encoded_region_name = 1;
   required bytes table_name = 2;
   required uint64 log_sequence_number = 3;
@@ -61,6 +62,14 @@ message WALEdit {
   optional uint64 nonceGroup = 9;
   optional uint64 nonce = 10;
   optional uint64 orig_sequence_number = 11;
+
+/*
+  optional CustomEntryType custom_entry_type = 9;
+
+  enum CustomEntryType {
+    COMPACTION = 0;
+  }
+*/
 }
 
 enum ScopeType {

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index 96272c7..0967e94 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -109,7 +109,7 @@ public class ReplicationProtbufUtil {
     for (Entry entry: entries) {
       entryBuilder.clear();
       // TODO: this duplicates a lot in WALKey#getBuilder
-      WALProtos.WALEdit.Builder keyBuilder = entryBuilder.getEditBuilder();
+      WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
       WALKey key = entry.getKey();
       keyBuilder.setEncodedRegionName(
           UnsafeByteOperations.unsafeWrap(encodedRegionName == null

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/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 61c725b..24281c7 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
@@ -597,13 +597,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * @param row
    * @param family
    * @param qualifier
-   * @param op
+   * @param compareOp
    * @param comparator @throws IOException
    */
   private boolean checkAndRowMutate(final Region region, final List<ClientProtos.Action> actions,
-      final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
-      CompareOperator op, ByteArrayComparable comparator, RegionActionResult.Builder builder,
-      ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
+                                    final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
+                                    CompareOperator op, ByteArrayComparable comparator, RegionActionResult.Builder builder,
+                                    ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
     if (!region.getRegionInfo().isMetaTable()) {
       regionServer.cacheFlusher.reclaimMemStoreMemory();
     }
@@ -648,6 +648,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
   /**
    * Execute an append mutation.
+   *
+   * @param region
+   * @param m
+   * @param cellScanner
    * @return result to return to client if default operation should be
    * bypassed as indicated by RegionObserver, null otherwise
    * @throws IOException
@@ -2073,7 +2077,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         // empty input
         return ReplicateWALEntryResponse.newBuilder().build();
       }
-      ByteString regionName = entries.get(0).getEdit().getEncodedRegionName();
+      ByteString regionName = entries.get(0).getKey().getEncodedRegionName();
       Region region = regionServer.getRegionByEncodedName(regionName.toStringUtf8());
       RegionCoprocessorHost coprocessorHost =
           ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())
@@ -2086,19 +2090,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       Durability durability = isPrimary ? Durability.USE_DEFAULT : Durability.SKIP_WAL;
 
       for (WALEntry entry : entries) {
-        if (!regionName.equals(entry.getEdit().getEncodedRegionName())) {
+        if (!regionName.equals(entry.getKey().getEncodedRegionName())) {
           throw new NotServingRegionException("Replay request contains entries from multiple " +
               "regions. First region:" + regionName.toStringUtf8() + " , other region:"
-              + entry.getEdit().getEncodedRegionName());
+              + entry.getKey().getEncodedRegionName());
         }
         if (regionServer.nonceManager != null && isPrimary) {
-          long nonceGroup = entry.getEdit().hasNonceGroup()
-            ? entry.getEdit().getNonceGroup() : HConstants.NO_NONCE;
-          long nonce = entry.getEdit().hasNonce() ? entry.getEdit().getNonce() : HConstants.NO_NONCE;
+          long nonceGroup = entry.getKey().hasNonceGroup()
+            ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
+          long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
           regionServer.nonceManager.reportOperationFromWal(
               nonceGroup,
               nonce,
-              entry.getEdit().getWriteTime());
+              entry.getKey().getWriteTime());
         }
         Pair<WALKey, WALEdit> walEntry = (coprocessorHost == null) ? null : new Pair<>();
         List<WALSplitter.MutationReplay> edits = WALSplitter.getMutationsFromWALEntry(entry,
@@ -2117,8 +2121,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           // HBASE-17924
           // sort to improve lock efficiency
           Collections.sort(edits);
-          long replaySeqId = (entry.getEdit().hasOrigSequenceNumber()) ?
-            entry.getEdit().getOrigSequenceNumber() : entry.getEdit().getLogSequenceNumber();
+          long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
+            entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
           OperationStatus[] result = doReplayBatchOp(region, edits, replaySeqId);
           // check if it's a partial success
           for (int i = 0; result != null && i < result.length; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
index 679e515..78c055e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader.Builder;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALEdit;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -333,7 +333,7 @@ public class ProtobufLogReader extends ReaderBase {
         }
         return false;
       }
-      WALEdit.Builder builder = WALEdit.newBuilder();
+      WALKey.Builder builder = WALKey.newBuilder();
       long size = 0;
       try {
         long available = -1;
@@ -363,12 +363,11 @@ public class ProtobufLogReader extends ReaderBase {
           throw new EOFException("Partial PB while reading WAL, " +
               "probably an unexpected EOF, ignoring. current offset=" + this.inputStream.getPos());
         }
-        WALEdit walKey = builder.build();
+        WALKey walKey = builder.build();
         entry.getKey().readFieldsFromPb(walKey, this.byteStringUncompressor);
         if (!walKey.hasFollowingKvCount() || 0 == walKey.getFollowingKvCount()) {
           if (LOG.isTraceEnabled()) {
-            LOG.trace("WALKey has no KVs that follow it; trying the next one. current offset=" +
-              this.inputStream.getPos());
+            LOG.trace("WALKey has no KVs that follow it; trying the next one. current offset=" + this.inputStream.getPos());
           }
           continue;
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 116f50f..0fbc74f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -159,7 +159,7 @@ public class ReplicationSink {
 
       for (WALEntry entry : entries) {
         TableName table =
-            TableName.valueOf(entry.getEdit().getTableName().toByteArray());
+            TableName.valueOf(entry.getKey().getTableName().toByteArray());
         Cell previousCell = null;
         Mutation m = null;
         int count = entry.getAssociatedCellCount();
@@ -183,8 +183,8 @@ public class ReplicationSink {
                   CellUtil.isDelete(cell) ? new Delete(cell.getRowArray(), cell.getRowOffset(),
                       cell.getRowLength()) : new Put(cell.getRowArray(), cell.getRowOffset(),
                       cell.getRowLength());
-              List<UUID> clusterIds = new ArrayList<>(entry.getEdit().getClusterIdsList().size());
-              for (HBaseProtos.UUID clusterId : entry.getEdit().getClusterIdsList()) {
+              List<UUID> clusterIds = new ArrayList<>(entry.getKey().getClusterIdsList().size());
+              for (HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()) {
                 clusterIds.add(toUUID(clusterId));
               }
               m.setClusterIds(clusterIds);
@@ -221,7 +221,7 @@ public class ReplicationSink {
       }
 
       int size = entries.size();
-      this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getEdit().getWriteTime());
+      this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());
       this.metrics.applyBatch(size + hfilesReplicated, hfilesReplicated);
       this.totalReplicatedEdits.addAndGet(totalReplicated);
     } catch (IOException ex) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index 9c696af..fd40ec4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -539,9 +539,9 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     this.encodedRegionName = encodedRegionName;
   }
 
-  public WALProtos.WALEdit.Builder getBuilder(
+  public WALProtos.WALKey.Builder getBuilder(
       WALCellCodec.ByteStringCompressor compressor) throws IOException {
-    WALProtos.WALEdit.Builder builder = WALProtos.WALEdit.newBuilder();
+    WALProtos.WALKey.Builder builder = WALProtos.WALKey.newBuilder();
     if (compressionContext == null) {
       builder.setEncodedRegionName(UnsafeByteOperations.unsafeWrap(this.encodedRegionName));
       builder.setTableName(UnsafeByteOperations.unsafeWrap(this.tablename.getName()));
@@ -580,42 +580,42 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     return builder;
   }
 
-  public void readFieldsFromPb(WALProtos.WALEdit walEdit,
+  public void readFieldsFromPb(WALProtos.WALKey walKey,
                                WALCellCodec.ByteStringUncompressor uncompressor)
       throws IOException {
     if (this.compressionContext != null) {
       this.encodedRegionName = uncompressor.uncompress(
-          walEdit.getEncodedRegionName(), compressionContext.regionDict);
+          walKey.getEncodedRegionName(), compressionContext.regionDict);
       byte[] tablenameBytes = uncompressor.uncompress(
-      walEdit.getTableName(), compressionContext.tableDict);
+          walKey.getTableName(), compressionContext.tableDict);
       this.tablename = TableName.valueOf(tablenameBytes);
     } else {
-      this.encodedRegionName = walEdit.getEncodedRegionName().toByteArray();
-      this.tablename = TableName.valueOf(walEdit.getTableName().toByteArray());
+      this.encodedRegionName = walKey.getEncodedRegionName().toByteArray();
+      this.tablename = TableName.valueOf(walKey.getTableName().toByteArray());
     }
     clusterIds.clear();
-    for (HBaseProtos.UUID clusterId : walEdit.getClusterIdsList()) {
+    for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
       clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
     }
-    if (walEdit.hasNonceGroup()) {
-      this.nonceGroup = walEdit.getNonceGroup();
+    if (walKey.hasNonceGroup()) {
+      this.nonceGroup = walKey.getNonceGroup();
     }
-    if (walEdit.hasNonce()) {
-      this.nonce = walEdit.getNonce();
+    if (walKey.hasNonce()) {
+      this.nonce = walKey.getNonce();
     }
     this.replicationScope = null;
-    if (walEdit.getScopesCount() > 0) {
+    if (walKey.getScopesCount() > 0) {
       this.replicationScope = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for (FamilyScope scope : walEdit.getScopesList()) {
+      for (FamilyScope scope : walKey.getScopesList()) {
         byte[] family = (compressionContext == null) ? scope.getFamily().toByteArray() :
           uncompressor.uncompress(scope.getFamily(), compressionContext.familyDict);
         this.replicationScope.put(family, scope.getScopeType().getNumber());
       }
     }
-    setSequenceId(walEdit.getLogSequenceNumber());
-    this.writeTime = walEdit.getWriteTime();
-    if(walEdit.hasOrigSequenceNumber()) {
-      this.origLogSeqNum = walEdit.getOrigSequenceNumber();
+    setSequenceId(walKey.getLogSequenceNumber());
+    this.writeTime = walKey.getWriteTime();
+    if(walKey.hasOrigSequenceNumber()) {
+      this.origLogSeqNum = walKey.getOrigSequenceNumber();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 6a7d896..aeacd9d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -2337,8 +2337,8 @@ public class WALSplitter {
       return new ArrayList<>();
     }
 
-    long replaySeqId = (entry.getEdit().hasOrigSequenceNumber()) ?
-      entry.getEdit().getOrigSequenceNumber() : entry.getEdit().getLogSequenceNumber();
+    long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
+      entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
     int count = entry.getAssociatedCellCount();
     List<MutationReplay> mutations = new ArrayList<>();
     Cell previousCell = null;
@@ -2368,9 +2368,9 @@ public class WALSplitter {
         } else {
           m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
           // Puts might come from increment or append, thus we need nonces.
-          long nonceGroup = entry.getEdit().hasNonceGroup()
-              ? entry.getEdit().getNonceGroup() : HConstants.NO_NONCE;
-          long nonce = entry.getEdit().hasNonce() ? entry.getEdit().getNonce() : HConstants.NO_NONCE;
+          long nonceGroup = entry.getKey().hasNonceGroup()
+              ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
+          long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
           mutations.add(new MutationReplay(MutationType.PUT, m, nonceGroup, nonce));
         }
       }
@@ -2385,10 +2385,10 @@ public class WALSplitter {
 
     // reconstruct WALKey
     if (logEntry != null) {
-      org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALEdit walKeyProto =
-          entry.getEdit();
+      org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey walKeyProto =
+          entry.getKey();
       List<UUID> clusterIds = new ArrayList<>(walKeyProto.getClusterIdsCount());
-      for (HBaseProtos.UUID uuid : entry.getEdit().getClusterIdsList()) {
+      for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
         clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
       }
       key = new WALKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(

http://git-wip-us.apache.org/repos/asf/hbase/blob/37696fff/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index 3ce59a4..dc0ca08 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -375,7 +376,7 @@ public class TestReplicationSink {
   private WALEntry.Builder createWALEntryBuilder(TableName table) {
     WALEntry.Builder builder = WALEntry.newBuilder();
     builder.setAssociatedCellCount(1);
-    WALProtos.WALEdit.Builder keyBuilder = WALProtos.WALEdit.newBuilder();
+    WALKey.Builder keyBuilder = WALKey.newBuilder();
     UUID.Builder uuidBuilder = UUID.newBuilder();
     uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
     uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
@@ -384,7 +385,7 @@ public class TestReplicationSink {
     keyBuilder.setWriteTime(System.currentTimeMillis());
     keyBuilder.setEncodedRegionName(UnsafeByteOperations.unsafeWrap(HConstants.EMPTY_BYTE_ARRAY));
     keyBuilder.setLogSequenceNumber(-1);
-    builder.setEdit(keyBuilder.build());
+    builder.setKey(keyBuilder.build());
     return builder;
   }
 }


[38/47] hbase git commit: HBASE-18823 Apply RegionInfo to MasterObserver/RegionObserver/WALObserver

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
index 96d54a5..b168be7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.junit.Assert.*;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
@@ -31,20 +29,21 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 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.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -54,7 +53,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 @Category(MediumTests.class)
 public class TestRegionObserverForAddingMutationsFromCoprocessors {
@@ -272,7 +272,7 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
     static WALEdit savedEdit = null;
     @Override
     public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-        HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+                             RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
       if (info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
         savedEdit = logEdit;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 979936f..c3627f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -18,13 +18,6 @@
  */
 package org.apache.hadoop.hbase.namespace;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
@@ -54,6 +47,7 @@ import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -87,6 +81,13 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 
+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;
+
 @Category(MediumTests.class)
 public class TestNamespaceAuditor {
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
@@ -303,7 +304,7 @@ public class TestNamespaceAuditor {
     @Override
     public synchronized void preMergeRegionsAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge) throws IOException {
+        final RegionInfo[] regionsToMerge) throws IOException {
       notifyAll();
       if (shouldFailMerge) {
         throw new IOException("fail merge");
@@ -540,7 +541,7 @@ public class TestNamespaceAuditor {
 
     @Override
     public void preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       if (throwExceptionInPreCreateTableAction) {
         throw new IOException("Throw exception as it is demanded.");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 1fee7ca..cfb0ce0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -17,12 +17,6 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion;
-import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 import java.io.IOException;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -39,6 +33,7 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.Table;
@@ -47,22 +42,23 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.WALObserver;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint.RegionReplicaReplayCallable;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -72,7 +68,11 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion;
+import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests RegionReplicaReplicationEndpoint. Unlike TestRegionReplicaReplicationEndpoint this
@@ -154,7 +154,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
     }
     @Override
     public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-        HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+                             RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
       // only keep primary region's edits
       if (logKey.getTablename().equals(tableName) && info.getReplicaId() == 0) {
         entries.add(new Entry(logKey, logEdit));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index 19cbc38..afb14fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -18,9 +18,6 @@
 
 package org.apache.hadoop.hbase.security.access;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedActionException;
@@ -30,30 +27,32 @@ import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-import org.apache.hadoop.hbase.client.Admin;
 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.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
@@ -63,12 +62,12 @@ import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.ServiceException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * Utility methods for testing security
@@ -630,7 +629,7 @@ public class SecureTestUtil {
     @Override
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()
       if (tableCreationLatch != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index d1c287f..58ddc2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -18,9 +18,6 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -40,36 +37,37 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 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.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
 import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
@@ -77,6 +75,9 @@ import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
 import org.apache.zookeeper.KeeperException;
 import org.junit.rules.TestName;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 /**
  * This is the base class for  HBaseFsck's ability to detect reasons for inconsistent tables.
  *
@@ -601,7 +602,7 @@ public class BaseTestHBaseFsck {
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final TableDescriptor desc,
-        final HRegionInfo[] regions) throws IOException {
+        final RegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()
       if (tableCreationLatch != null) {


[39/47] hbase git commit: HBASE-18823 Apply RegionInfo to MasterObserver/RegionObserver/WALObserver

Posted by bu...@apache.org.
HBASE-18823 Apply RegionInfo to MasterObserver/RegionObserver/WALObserver


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

Branch: refs/heads/HBASE-18467
Commit: a6c3c645fd193f1805d449d91528e2a9246f8ff8
Parents: 37696ff
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Tue Sep 19 01:21:55 2017 +0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Wed Sep 20 18:43:11 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/BackupObserver.java     |  8 +--
 .../ExampleMasterObserverWithMetrics.java       |  8 +--
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     |  9 ++-
 .../hbase/coprocessor/MasterObserver.java       | 60 +++++++++----------
 .../RegionCoprocessorEnvironment.java           |  8 +--
 .../hbase/coprocessor/RegionObserver.java       | 14 ++---
 .../hadoop/hbase/coprocessor/WALObserver.java   | 10 ++--
 .../hbase/master/MasterCoprocessorHost.java     | 56 +++++++++---------
 .../regionserver/RegionCoprocessorHost.java     | 27 ++++-----
 .../hbase/security/access/AccessController.java | 61 ++++++++++----------
 .../CoprocessorWhitelistMasterObserver.java     | 16 +++--
 .../hadoop/hbase/client/TestEnableTable.java    | 15 +++--
 .../coprocessor/SampleRegionWALObserver.java    | 12 ++--
 .../hbase/coprocessor/SimpleRegionObserver.java | 22 ++++---
 .../coprocessor/TestCoprocessorMetrics.java     | 31 +++++-----
 ...TestMasterCoprocessorExceptionWithAbort.java | 14 ++---
 ...estMasterCoprocessorExceptionWithRemove.java | 12 ++--
 .../hbase/coprocessor/TestMasterObserver.java   | 61 ++++++++++----------
 ...erverForAddingMutationsFromCoprocessors.java | 12 ++--
 .../hbase/namespace/TestNamespaceAuditor.java   | 19 +++---
 ...egionReplicaReplicationEndpointNoMaster.java | 20 +++----
 .../hbase/security/access/SecureTestUtil.java   | 21 ++++---
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    | 19 +++---
 23 files changed, 263 insertions(+), 272 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
index 98d507d..4131b4d 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
@@ -27,17 +27,17 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupManager;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * An Observer to facilitate backup operations
@@ -61,7 +61,7 @@ public class BackupObserver implements RegionObserver {
     try (Connection connection = ConnectionFactory.createConnection(cfg);
         BackupSystemTable tbl = new BackupSystemTable(connection)) {
       List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
+      RegionInfo info = ctx.getEnvironment().getRegionInfo();
       TableName tableName = info.getTable();
       if (!fullyBackedUpTables.contains(tableName)) {
         if (LOG.isTraceEnabled()) {
@@ -87,7 +87,7 @@ public class BackupObserver implements RegionObserver {
     try (Connection connection = ConnectionFactory.createConnection(cfg);
         BackupSystemTable tbl = new BackupSystemTable(connection)) {
       List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
+      RegionInfo info = ctx.getEnvironment().getRegionInfo();
       TableName tableName = info.getTable();
       if (!fullyBackedUpTables.contains(tableName)) {
         if (LOG.isTraceEnabled()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
index 8535d05..a93935d 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
@@ -23,11 +23,11 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.metrics.Counter;
 import org.apache.hadoop.hbase.metrics.Gauge;
@@ -68,7 +68,7 @@ public class ExampleMasterObserverWithMetrics implements MasterObserver {
 
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                             TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                             TableDescriptor desc, RegionInfo[] regions) throws IOException {
     // we rely on the fact that there is only 1 instance of our MasterObserver. We keep track of
     // when the operation starts before the operation is executing.
     this.createTableStartTime = System.currentTimeMillis();
@@ -76,7 +76,7 @@ public class ExampleMasterObserverWithMetrics implements MasterObserver {
 
   @Override
   public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                              TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                              TableDescriptor desc, RegionInfo[] regions) throws IOException {
     if (this.createTableStartTime > 0) {
       long time = System.currentTimeMillis() - this.createTableStartTime;
       LOG.info("Create table took: " + time);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index a78d33e..68bde3f 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
@@ -30,11 +29,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
@@ -68,7 +65,9 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveTablesR
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RSGroupAdminService;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService {
@@ -309,7 +308,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
   // Assign table to default RSGroup.
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+      TableDescriptor desc, RegionInfo[] regions) throws IOException {
     assignTableToGroup(desc);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 9d649cc..64b54fc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -25,16 +25,14 @@ import java.util.Set;
 
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
@@ -47,6 +45,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 
 /**
@@ -88,7 +88,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regions the initial regions created for the table
    */
   default void preCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableDescriptor desc, HRegionInfo[] regions) throws IOException {}
+      TableDescriptor desc, RegionInfo[] regions) throws IOException {}
 
   /**
    * Called after the createTable operation has been requested.  Called as part
@@ -98,7 +98,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regions the initial regions created for the table
    */
   default void postCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableDescriptor desc, HRegionInfo[] regions) throws IOException {}
+      TableDescriptor desc, RegionInfo[] regions) throws IOException {}
 
   /**
    * Called before a new table is created by
@@ -113,7 +113,7 @@ public interface MasterObserver extends Coprocessor {
   default void preCreateTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableDescriptor desc,
-      final HRegionInfo[] regions) throws IOException {}
+      final RegionInfo[] regions) throws IOException {}
 
   /**
    * Called after the createTable operation has been requested.  Called as part
@@ -127,7 +127,7 @@ public interface MasterObserver extends Coprocessor {
   default void postCompletedCreateTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableDescriptor desc,
-      final HRegionInfo[] regions) throws IOException {}
+      final RegionInfo[] regions) throws IOException {}
 
   /**
    * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
@@ -550,24 +550,24 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called prior to moving a given region from one region server to another.
    * @param ctx the environment to interact with the framework and master
-   * @param region the HRegionInfo
+   * @param region the RegionInfo
    * @param srcServer the source ServerName
    * @param destServer the destination ServerName
    */
   default void preMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo region, final ServerName srcServer,
+      final RegionInfo region, final ServerName srcServer,
       final ServerName destServer)
     throws IOException {}
 
   /**
    * Called after the region move has been requested.
    * @param ctx the environment to interact with the framework and master
-   * @param region the HRegionInfo
+   * @param region the RegionInfo
    * @param srcServer the source ServerName
    * @param destServer the destination ServerName
    */
   default void postMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo region, final ServerName srcServer,
+      final RegionInfo region, final ServerName srcServer,
       final ServerName destServer)
     throws IOException {}
 
@@ -577,7 +577,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regionInfo the regionInfo of the region
    */
   default void preAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo regionInfo) throws IOException {}
+      final RegionInfo regionInfo) throws IOException {}
 
   /**
    * Called after the region assignment has been requested.
@@ -585,7 +585,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regionInfo the regionInfo of the region
    */
   default void postAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo regionInfo) throws IOException {}
+      final RegionInfo regionInfo) throws IOException {}
 
   /**
    * Called prior to unassigning a given region.
@@ -594,7 +594,7 @@ public interface MasterObserver extends Coprocessor {
    * @param force whether to force unassignment or not
    */
   default void preUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo regionInfo, final boolean force) throws IOException {}
+      final RegionInfo regionInfo, final boolean force) throws IOException {}
 
   /**
    * Called after the region unassignment has been requested.
@@ -603,7 +603,7 @@ public interface MasterObserver extends Coprocessor {
    * @param force whether to force unassignment or not
    */
   default void postUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo regionInfo, final boolean force) throws IOException {}
+      final RegionInfo regionInfo, final boolean force) throws IOException {}
 
   /**
    * Called prior to marking a given region as offline. <code>ctx.bypass()</code> will not have any
@@ -612,7 +612,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regionInfo
    */
   default void preRegionOffline(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo regionInfo) throws IOException {}
+      final RegionInfo regionInfo) throws IOException {}
 
   /**
    * Called after the region has been marked offline.
@@ -620,7 +620,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regionInfo
    */
   default void postRegionOffline(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo regionInfo) throws IOException {}
+      final RegionInfo regionInfo) throws IOException {}
 
   /**
    * Called prior to requesting rebalancing of the cluster regions, though after
@@ -692,8 +692,8 @@ public interface MasterObserver extends Coprocessor {
    */
   default void postCompletedSplitRegionAction(
       final ObserverContext<MasterCoprocessorEnvironment> c,
-      final HRegionInfo regionInfoA,
-      final HRegionInfo regionInfoB) throws IOException {}
+      final RegionInfo regionInfoA,
+      final RegionInfo regionInfoB) throws IOException {}
 
   /**
    * This will be called before PONR step as part of split transaction. Calling
@@ -733,7 +733,7 @@ public interface MasterObserver extends Coprocessor {
    */
   default void preMergeRegionsAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo[] regionsToMerge) throws IOException {}
+      final RegionInfo[] regionsToMerge) throws IOException {}
 
   /**
    * called after the regions merge.
@@ -741,8 +741,8 @@ public interface MasterObserver extends Coprocessor {
    */
   default void postCompletedMergeRegionsAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo[] regionsToMerge,
-      final HRegionInfo mergedRegion) throws IOException {}
+      final RegionInfo[] regionsToMerge,
+      final RegionInfo mergedRegion) throws IOException {}
 
   /**
    * This will be called before PONR step as part of regions merge transaction. Calling
@@ -753,7 +753,7 @@ public interface MasterObserver extends Coprocessor {
    */
   default void preMergeRegionsCommitAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo[] regionsToMerge,
+      final RegionInfo[] regionsToMerge,
       @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException {}
 
   /**
@@ -762,8 +762,8 @@ public interface MasterObserver extends Coprocessor {
    */
   default void postMergeRegionsCommitAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo[] regionsToMerge,
-      final HRegionInfo mergedRegion) throws IOException {}
+      final RegionInfo[] regionsToMerge,
+      final RegionInfo mergedRegion) throws IOException {}
 
   /**
    * This will be called after the roll back of the regions merge.
@@ -771,7 +771,7 @@ public interface MasterObserver extends Coprocessor {
    */
   default void postRollBackMergeRegionsAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo[] regionsToMerge) throws IOException {}
+      final RegionInfo[] regionsToMerge) throws IOException {}
 
   /**
    * Called prior to modifying the flag used to enable/disable region balancing.
@@ -1173,7 +1173,7 @@ public interface MasterObserver extends Coprocessor {
    */
   default void preMergeRegions(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HRegionInfo[] regionsToMerge) throws IOException {}
+      final RegionInfo[] regionsToMerge) throws IOException {}
 
   /**
    * called after merge regions request.
@@ -1182,7 +1182,7 @@ public interface MasterObserver extends Coprocessor {
    */
   default void postMergeRegions(
       final ObserverContext<MasterCoprocessorEnvironment> c,
-      final HRegionInfo[] regionsToMerge) throws IOException {}
+      final RegionInfo[] regionsToMerge) throws IOException {}
 
   /**
    * Called before servers are moved to target region server group
@@ -1407,7 +1407,7 @@ public interface MasterObserver extends Coprocessor {
    * @param ctx the environment to interact with the framework and master
    */
   default void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
-      TableName tableName, HRegionInfo[] regionInfos, LockType type,
+      TableName tableName, RegionInfo[] regionInfos, LockType type,
       String description) throws IOException {}
 
   /**
@@ -1415,7 +1415,7 @@ public interface MasterObserver extends Coprocessor {
    * @param ctx the environment to interact with the framework and master
    */
   default void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
-      TableName tableName, HRegionInfo[] regionInfos, LockType type,
+      TableName tableName, RegionInfo[] regionInfos, LockType type,
       String description) throws IOException {}
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
index 2d99a80..2a3d62c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
@@ -23,12 +23,12 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 @InterfaceStability.Evolving
@@ -37,7 +37,7 @@ public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment {
   Region getRegion();
 
   /** @return region information for the region this coprocessor is running on */
-  HRegionInfo getRegionInfo();
+  RegionInfo getRegionInfo();
 
   /** @return reference to the region server services */
   RegionServerServices getRegionServerServices();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/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 9f16c7c..9318b9a 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
@@ -30,9 +30,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -40,6 +37,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -63,6 +61,8 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * Coprocessors implement this interface to observe and mediate client actions on the region.
@@ -943,7 +943,7 @@ public interface RegionObserver extends Coprocessor {
    * @param edits the file of recovered edits
    */
   default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
-      HRegionInfo info, Path edits) throws IOException {}
+    RegionInfo info, Path edits) throws IOException {}
 
   /**
    * Called after replaying WALs for this region.
@@ -952,7 +952,7 @@ public interface RegionObserver extends Coprocessor {
    * @param edits the file of recovered edits
    */
   default void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
-      HRegionInfo info, Path edits) throws IOException {}
+    RegionInfo info, Path edits) throws IOException {}
 
   /**
    * Called before a {@link WALEdit}
@@ -960,7 +960,7 @@ public interface RegionObserver extends Coprocessor {
    * @param ctx the environment provided by the region server
    */
   default void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
+    RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 
   /**
    * Called after a {@link WALEdit}
@@ -968,7 +968,7 @@ public interface RegionObserver extends Coprocessor {
    * @param ctx the environment provided by the region server
    */
   default void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
+    RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 
   /**
    * Called before bulkLoadHFile. Users can create a StoreFile instance to

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
index a071a31..52c27f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
@@ -24,11 +24,11 @@ import java.io.IOException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * It's provided to have a way for coprocessors to observe, rewrite,
@@ -75,7 +75,7 @@ public interface WALObserver extends Coprocessor {
    */
   // TODO: return value is not used
   default boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
     return false;
   }
 
@@ -84,7 +84,7 @@ public interface WALObserver extends Coprocessor {
    * is writen to WAL.
    */
   default void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
+      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 
   /**
    * Called before rolling the current WAL

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index f0ce714..badcfed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -28,15 +28,14 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
@@ -57,6 +56,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Provides the coprocessor framework and environment for master oriented
@@ -242,7 +242,7 @@ public class MasterCoprocessorHost
 
   /* Implementation of hooks for invoking MasterObservers */
 
-  public void preCreateTable(final TableDescriptor htd, final HRegionInfo[] regions)
+  public void preCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -253,7 +253,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postCreateTable(final TableDescriptor htd, final HRegionInfo[] regions)
+  public void postCreateTable(final TableDescriptor htd, final RegionInfo[] regions)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -264,7 +264,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preCreateTableAction(final TableDescriptor htd, final HRegionInfo[] regions,
+  public void preCreateTableAction(final TableDescriptor htd, final RegionInfo[] regions,
                                    final User user)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@@ -277,7 +277,7 @@ public class MasterCoprocessorHost
   }
 
   public void postCompletedCreateTableAction(
-      final TableDescriptor htd, final HRegionInfo[] regions, final User user) throws IOException {
+      final TableDescriptor htd, final RegionInfo[] regions, final User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -704,8 +704,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public boolean preMove(final HRegionInfo region, final ServerName srcServer,
-      final ServerName destServer) throws IOException {
+  public boolean preMove(final RegionInfo region, final ServerName srcServer,
+                         final ServerName destServer) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -715,7 +715,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postMove(final HRegionInfo region, final ServerName srcServer,
+  public void postMove(final RegionInfo region, final ServerName srcServer,
       final ServerName destServer) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -726,7 +726,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public boolean preAssign(final HRegionInfo regionInfo) throws IOException {
+  public boolean preAssign(final RegionInfo regionInfo) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -736,7 +736,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postAssign(final HRegionInfo regionInfo) throws IOException {
+  public void postAssign(final RegionInfo regionInfo) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -746,7 +746,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public boolean preUnassign(final HRegionInfo regionInfo, final boolean force)
+  public boolean preUnassign(final RegionInfo regionInfo, final boolean force)
       throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -757,7 +757,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postUnassign(final HRegionInfo regionInfo, final boolean force) throws IOException {
+  public void postUnassign(final RegionInfo regionInfo, final boolean force) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -767,7 +767,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preRegionOffline(final HRegionInfo regionInfo) throws IOException {
+  public void preRegionOffline(final RegionInfo regionInfo) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -777,7 +777,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postRegionOffline(final HRegionInfo regionInfo) throws IOException {
+  public void postRegionOffline(final RegionInfo regionInfo) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -787,7 +787,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preMergeRegions(final HRegionInfo[] regionsToMerge)
+  public void preMergeRegions(final RegionInfo[] regionsToMerge)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -798,7 +798,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postMergeRegions(final HRegionInfo[] regionsToMerge)
+  public void postMergeRegions(final RegionInfo[] regionsToMerge)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -897,8 +897,8 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public void postCompletedSplitRegionAction(
-      final HRegionInfo regionInfoA,
-      final HRegionInfo regionInfoB,
+      final RegionInfo regionInfoA,
+      final RegionInfo regionInfoB,
       final User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
@@ -966,7 +966,7 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public boolean preMergeRegionsAction(
-      final HRegionInfo[] regionsToMerge, final User user) throws IOException {
+      final RegionInfo[] regionsToMerge, final User user) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver,
@@ -984,8 +984,8 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public void postCompletedMergeRegionsAction(
-      final HRegionInfo[] regionsToMerge,
-      final HRegionInfo mergedRegion,
+      final RegionInfo[] regionsToMerge,
+      final RegionInfo mergedRegion,
       final User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
@@ -1004,7 +1004,7 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public boolean preMergeRegionsCommit(
-      final HRegionInfo[] regionsToMerge,
+      final RegionInfo[] regionsToMerge,
       final @MetaMutationAnnotation List<Mutation> metaEntries,
       final User user) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@@ -1024,8 +1024,8 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public void postMergeRegionsCommit(
-      final HRegionInfo[] regionsToMerge,
-      final HRegionInfo mergedRegion,
+      final RegionInfo[] regionsToMerge,
+      final RegionInfo mergedRegion,
       final User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
@@ -1043,7 +1043,7 @@ public class MasterCoprocessorHost
    * @throws IOException
    */
   public void postRollBackMergeRegionsAction(
-      final HRegionInfo[] regionsToMerge, final User user) throws IOException {
+      final RegionInfo[] regionsToMerge, final User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
       @Override
       public void call(MasterObserver oserver,
@@ -1809,7 +1809,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preRequestLock(String namespace, TableName tableName, HRegionInfo[] regionInfos,
+  public void preRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
       LockType type, String description) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -1820,7 +1820,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postRequestLock(String namespace, TableName tableName, HRegionInfo[] regionInfos,
+  public void postRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
       LockType type, String description) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/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 f282766..044c4dc 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
@@ -19,9 +19,6 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -32,6 +29,8 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.regex.Matcher;
 
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
 import org.apache.commons.collections4.map.AbstractReferenceMap;
 import org.apache.commons.collections4.map.ReferenceMap;
 import org.apache.commons.lang3.ClassUtils;
@@ -46,9 +45,6 @@ import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -56,6 +52,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -75,17 +72,17 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * Implements the coprocessor environment and runtime support for coprocessors
@@ -156,7 +153,7 @@ public class RegionCoprocessorHost
     }
 
     @Override
-    public HRegionInfo getRegionInfo() {
+    public RegionInfo getRegionInfo() {
       return region.getRegionInfo();
     }
 
@@ -1264,7 +1261,7 @@ public class RegionCoprocessorHost
    * @param edits the file of recovered edits
    * @throws IOException Exception
    */
-  public void preReplayWALs(final HRegionInfo info, final Path edits) throws IOException {
+  public void preReplayWALs(final RegionInfo info, final Path edits) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -1279,7 +1276,7 @@ public class RegionCoprocessorHost
    * @param edits the file of recovered edits
    * @throws IOException Exception
    */
-  public void postReplayWALs(final HRegionInfo info, final Path edits) throws IOException {
+  public void postReplayWALs(final RegionInfo info, final Path edits) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -1296,7 +1293,7 @@ public class RegionCoprocessorHost
    * @return true if default behavior should be bypassed, false otherwise
    * @throws IOException
    */
-  public boolean preWALRestore(final HRegionInfo info, final WALKey logKey,
+  public boolean preWALRestore(final RegionInfo info, final WALKey logKey,
       final WALEdit logEdit) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
       @Override
@@ -1313,7 +1310,7 @@ public class RegionCoprocessorHost
    * @param logEdit
    * @throws IOException
    */
-  public void postWALRestore(final HRegionInfo info, final WALKey logKey, final WALEdit logEdit)
+  public void postWALRestore(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/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 4c92fb1..e9bed7e 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
@@ -18,6 +18,20 @@
  */
 package org.apache.hadoop.hbase.security.access;
 
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
@@ -35,7 +49,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -43,7 +56,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
@@ -54,6 +66,7 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -93,8 +106,6 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
@@ -120,21 +131,9 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Provides basic authorization checks for data access and administrative
@@ -303,7 +302,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   AuthResult permissionGranted(String request, User user, Action permRequest,
       RegionCoprocessorEnvironment e,
       Map<byte [], ? extends Collection<?>> families) {
-    HRegionInfo hri = e.getRegion().getRegionInfo();
+    RegionInfo hri = e.getRegion().getRegionInfo();
     TableName tableName = hri.getTable();
 
     // 1. All users need read access to hbase:meta table.
@@ -645,7 +644,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
       RegionCoprocessorEnvironment env,
       Map<byte[], ? extends Collection<byte[]>> familyMap)
     throws IOException {
-    HRegionInfo hri = env.getRegion().getRegionInfo();
+    RegionInfo hri = env.getRegion().getRegionInfo();
     TableName tableName = hri.getTable();
 
     if (user == null) {
@@ -988,7 +987,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
-      TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+      TableDescriptor desc, RegionInfo[] regions) throws IOException {
     Set<byte[]> families = desc.getColumnFamilyNames();
     Map<byte[], Set<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     for (byte[] family: families) {
@@ -1002,7 +1001,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   public void postCompletedCreateTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> c,
       final TableDescriptor desc,
-      final HRegionInfo[] regions) throws IOException {
+      final RegionInfo[] regions) throws IOException {
     // When AC is used, it should be configured as the 1st CP.
     // In Master, the table operations like create, are handled by a Thread pool but the max size
     // for this pool is 1. So if multiple CPs create tables on startup, these creations will happen
@@ -1251,26 +1250,26 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   }
 
   @Override
-  public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo region,
+  public void preMove(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo region,
       ServerName srcServer, ServerName destServer) throws IOException {
     requirePermission(getActiveUser(c), "move", region.getTable(), null, null, Action.ADMIN);
   }
 
   @Override
-  public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo)
+  public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo)
       throws IOException {
     requirePermission(getActiveUser(c), "assign", regionInfo.getTable(), null, null, Action.ADMIN);
   }
 
   @Override
-  public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, HRegionInfo regionInfo,
+  public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfo,
       boolean force) throws IOException {
     requirePermission(getActiveUser(c), "unassign", regionInfo.getTable(), null, null, Action.ADMIN);
   }
 
   @Override
   public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> c,
-      HRegionInfo regionInfo) throws IOException {
+      RegionInfo regionInfo) throws IOException {
     requirePermission(getActiveUser(c), "regionOffline", regionInfo.getTable(), null, null,
         Action.ADMIN);
   }
@@ -1478,7 +1477,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
     if (region == null) {
       LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()");
     } else {
-      HRegionInfo regionInfo = region.getRegionInfo();
+      RegionInfo regionInfo = region.getRegionInfo();
       if (regionInfo.getTable().isSystemTable()) {
         checkSystemOrSuperUser(getActiveUser(c));
       } else {
@@ -2467,7 +2466,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   }
 
   private TableName getTableName(Region region) {
-    HRegionInfo regionInfo = region.getRegionInfo();
+    RegionInfo regionInfo = region.getRegionInfo();
     if (regionInfo != null) {
       return regionInfo.getTable();
     }
@@ -2569,7 +2568,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preMergeRegions(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-                              final HRegionInfo[] regionsToMerge) throws IOException {
+                              final RegionInfo[] regionsToMerge) throws IOException {
     requirePermission(getActiveUser(ctx), "mergeRegions", regionsToMerge[0].getTable(), null, null,
       Action.ADMIN);
   }
@@ -2718,7 +2717,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
-      TableName tableName, HRegionInfo[] regionInfos, LockType type, String description)
+      TableName tableName, RegionInfo[] regionInfos, LockType type, String description)
   throws IOException {
     // There are operations in the CREATE and ADMIN domain which may require lock, READ
     // or WRITE. So for any lock request, we check for these two perms irrespective of lock type.
@@ -2734,7 +2733,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   }
 
   private void checkLockPermissions(User user, String namespace,
-      TableName tableName, HRegionInfo[] regionInfos, String reason)
+      TableName tableName, RegionInfo[] regionInfos, String reason)
   throws IOException {
     if (namespace != null && !namespace.isEmpty()) {
       requireNamespacePermission(user, reason, namespace, Action.ADMIN, Action.CREATE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
index 52da2ae..0b765d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
@@ -25,21 +25,19 @@ import java.util.regex.Matcher;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Master observer for restricting coprocessor assignments.
@@ -61,7 +59,7 @@ public class CoprocessorWhitelistMasterObserver implements MasterObserver {
 
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableDescriptor htd, HRegionInfo[] regions) throws IOException {
+      TableDescriptor htd, RegionInfo[] regions) throws IOException {
     verifyCoprocessors(ctx, htd);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
index 133b111..379ab31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
@@ -18,10 +18,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -42,6 +38,9 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -53,9 +52,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 @Category({ MasterTests.class, MediumTests.class })
 public class TestEnableTable {
@@ -198,7 +197,7 @@ public class TestEnableTable {
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final TableDescriptor desc,
-        final HRegionInfo[] regions) throws IOException {
+        final RegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()
       if (tableCreationLatch != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java
index 8b0ad5f..e338941 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALObserver.java
@@ -28,10 +28,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 
 /**
@@ -83,13 +83,13 @@ public class SampleRegionWALObserver implements WALObserver, RegionObserver {
 
   @Override
   public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
     postWALWriteCalled = true;
   }
 
   @Override
   public boolean preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
     boolean bypass = false;
     // check table name matches or not.
     if (!Bytes.equals(info.getTable().toBytes(), this.tableName)) {
@@ -132,7 +132,7 @@ public class SampleRegionWALObserver implements WALObserver, RegionObserver {
    */
   @Override
   public void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+    RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
     preWALRestoreCalled = true;
   }
 
@@ -154,7 +154,7 @@ public class SampleRegionWALObserver implements WALObserver, RegionObserver {
    */
   @Override
   public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
     postWALRestoreCalled = true;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
index 023f8fc..9938c18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
@@ -19,11 +19,6 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -37,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -45,6 +39,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -62,13 +57,16 @@ import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * A sample region observer that tests the RegionObserver interface.
@@ -619,19 +617,19 @@ public class SimpleRegionObserver implements RegionObserver {
 
   @Override
   public void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> env,
-      HRegionInfo info, Path edits) throws IOException {
+      RegionInfo info, Path edits) throws IOException {
     ctPreReplayWALs.incrementAndGet();
   }
 
   @Override
   public void postReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> env,
-      HRegionInfo info, Path edits) throws IOException {
+      RegionInfo info, Path edits) throws IOException {
     ctPostReplayWALs.incrementAndGet();
   }
 
   @Override
   public void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
-      HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
     String tableName = logKey.getTablename().getNameAsString();
     if (tableName.equals(TABLE_SKIPPED)) {
       // skip recovery of TABLE_SKIPPED for testing purpose
@@ -643,7 +641,7 @@ public class SimpleRegionObserver implements RegionObserver {
 
   @Override
   public void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
-                             HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+                             RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
     ctPostWALRestore.incrementAndGet();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
index a823a8a..b4e10d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -18,16 +18,13 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.List;
 import java.util.Optional;
 
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -35,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -45,6 +41,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -61,23 +58,25 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRo
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
 import org.junit.rules.TestName;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Testing of coprocessor metrics end-to-end.
  */
@@ -102,14 +101,14 @@ public class TestCoprocessorMetrics {
 
     @Override
     public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                               TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                               TableDescriptor desc, RegionInfo[] regions) throws IOException {
       // we rely on the fact that there is only 1 instance of our MasterObserver
       this.start = System.currentTimeMillis();
     }
 
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                                TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                                TableDescriptor desc, RegionInfo[] regions) throws IOException {
       if (this.start > 0) {
         long time = System.currentTimeMillis() - start;
         LOG.info("Create table took: " + time);
@@ -162,7 +161,7 @@ public class TestCoprocessorMetrics {
 
     @Override
     public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-                             HRegionInfo info, org.apache.hadoop.hbase.wal.WALKey logKey,
+                             RegionInfo info, org.apache.hadoop.hbase.wal.WALKey logKey,
                              WALEdit logEdit) throws IOException {
       walEditsCount.increment();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
index c0a5801..ab3dec7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
@@ -19,11 +19,6 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -32,11 +27,11 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
 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.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -50,6 +45,11 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 /**
  * Tests unhandled exceptions thrown by coprocessors running on master.
  * Expected result is that the master will abort with an informative
@@ -105,7 +105,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
 
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       // cause a NullPointerException and don't catch it: this will cause the
       // master to abort().
       Integer i;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
index 92d12ee..ccd777f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
@@ -19,10 +19,6 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -30,11 +26,11 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 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.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -48,6 +44,10 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 /**
  * Tests unhandled exceptions thrown by coprocessors running on master.
  * Expected result is that the master will remove the buggy coprocessor from
@@ -82,7 +82,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
     @SuppressWarnings("null")
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       // Cause a NullPointerException and don't catch it: this should cause the
       // master to throw an o.apache.hadoop.hbase.DoNotRetryIOException to the
       // client.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6c3c645/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index fbfeb50..3becc61 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -19,11 +19,6 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
@@ -50,6 +45,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -82,6 +78,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 
 /**
  * Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.MasterObserver}
@@ -284,14 +285,14 @@ public class TestMasterObserver {
     @Override
     public void preMergeRegions(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge) throws IOException {
+        final RegionInfo[] regionsToMerge) throws IOException {
       preMergeRegionsCalled = true;
     }
 
     @Override
     public void postMergeRegions(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge) throws IOException {
+        final RegionInfo[] regionsToMerge) throws IOException {
       postMergeRegionsCalled = true;
     }
 
@@ -301,7 +302,7 @@ public class TestMasterObserver {
 
     @Override
     public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       if (bypass) {
         env.bypass();
       }
@@ -310,7 +311,7 @@ public class TestMasterObserver {
 
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       postCreateTableCalled = true;
     }
 
@@ -697,7 +698,7 @@ public class TestMasterObserver {
 
     @Override
     public void preMove(ObserverContext<MasterCoprocessorEnvironment> env,
-        HRegionInfo region, ServerName srcServer, ServerName destServer)
+        RegionInfo region, ServerName srcServer, ServerName destServer)
     throws IOException {
       if (bypass) {
         env.bypass();
@@ -706,7 +707,7 @@ public class TestMasterObserver {
     }
 
     @Override
-    public void postMove(ObserverContext<MasterCoprocessorEnvironment> env, HRegionInfo region,
+    public void postMove(ObserverContext<MasterCoprocessorEnvironment> env, RegionInfo region,
         ServerName srcServer, ServerName destServer)
     throws IOException {
       postMoveCalled = true;
@@ -722,7 +723,7 @@ public class TestMasterObserver {
 
     @Override
     public void preAssign(ObserverContext<MasterCoprocessorEnvironment> env,
-        final HRegionInfo regionInfo) throws IOException {
+        final RegionInfo regionInfo) throws IOException {
       if (bypass) {
         env.bypass();
       }
@@ -731,7 +732,7 @@ public class TestMasterObserver {
 
     @Override
     public void postAssign(ObserverContext<MasterCoprocessorEnvironment> env,
-        final HRegionInfo regionInfo) throws IOException {
+        final RegionInfo regionInfo) throws IOException {
       postAssignCalled = true;
     }
 
@@ -745,7 +746,7 @@ public class TestMasterObserver {
 
     @Override
     public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> env,
-        final HRegionInfo regionInfo, final boolean force) throws IOException {
+        final RegionInfo regionInfo, final boolean force) throws IOException {
       if (bypass) {
         env.bypass();
       }
@@ -754,7 +755,7 @@ public class TestMasterObserver {
 
     @Override
     public void postUnassign(ObserverContext<MasterCoprocessorEnvironment> env,
-        final HRegionInfo regionInfo, final boolean force) throws IOException {
+        final RegionInfo regionInfo, final boolean force) throws IOException {
       postUnassignCalled = true;
     }
 
@@ -768,13 +769,13 @@ public class TestMasterObserver {
 
     @Override
     public void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> env,
-        final HRegionInfo regionInfo) throws IOException {
+        final RegionInfo regionInfo) throws IOException {
       preRegionOfflineCalled = true;
     }
 
     @Override
     public void postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> env,
-        final HRegionInfo regionInfo) throws IOException {
+        final RegionInfo regionInfo) throws IOException {
       postRegionOfflineCalled = true;
     }
 
@@ -977,7 +978,7 @@ public class TestMasterObserver {
     public void preCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env,
         final TableDescriptor desc,
-        final HRegionInfo[] regions) throws IOException {
+        final RegionInfo[] regions) throws IOException {
       if (bypass) {
         env.bypass();
       }
@@ -988,7 +989,7 @@ public class TestMasterObserver {
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final TableDescriptor desc,
-        final HRegionInfo[] regions) throws IOException {
+        final RegionInfo[] regions) throws IOException {
       postCompletedCreateTableActionCalled = true;
       tableCreationLatch.countDown();
     }
@@ -1368,14 +1369,14 @@ public class TestMasterObserver {
 
     @Override
     public void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
-        TableName tableName, HRegionInfo[] regionInfos, LockType type,
+        TableName tableName, RegionInfo[] regionInfos, LockType type,
         String description) throws IOException {
       preRequestLockCalled = true;
     }
 
     @Override
     public void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
-        TableName tableName, HRegionInfo[] regionInfos, LockType type,
+        TableName tableName, RegionInfo[] regionInfos, LockType type,
         String description) throws IOException {
       postRequestLockCalled = true;
     }
@@ -1414,8 +1415,8 @@ public class TestMasterObserver {
     @Override
     public void postCompletedSplitRegionAction(
         final ObserverContext<MasterCoprocessorEnvironment> c,
-        final HRegionInfo regionInfoA,
-        final HRegionInfo regionInfoB) throws IOException {
+        final RegionInfo regionInfoA,
+        final RegionInfo regionInfoB) throws IOException {
     }
 
     @Override
@@ -1438,34 +1439,34 @@ public class TestMasterObserver {
     @Override
     public void preMergeRegionsAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge) throws IOException {
+        final RegionInfo[] regionsToMerge) throws IOException {
     }
 
     @Override
     public void postCompletedMergeRegionsAction(
         final ObserverContext<MasterCoprocessorEnvironment> c,
-        final HRegionInfo[] regionsToMerge,
-        final HRegionInfo mergedRegion) throws IOException {
+        final RegionInfo[] regionsToMerge,
+        final RegionInfo mergedRegion) throws IOException {
     }
 
     @Override
     public void preMergeRegionsCommitAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge,
+        final RegionInfo[] regionsToMerge,
         final List<Mutation> metaEntries) throws IOException {
     }
 
     @Override
     public void postMergeRegionsCommitAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge,
-        final HRegionInfo mergedRegion) throws IOException {
+        final RegionInfo[] regionsToMerge,
+        final RegionInfo mergedRegion) throws IOException {
     }
 
     @Override
     public void postRollBackMergeRegionsAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge) throws IOException {
+        final RegionInfo[] regionsToMerge) throws IOException {
     }
   }
 


[33/47] hbase git commit: HBASE-18849 expand "thirdparty" reference to give examples of setting netty location in common testing modules

Posted by bu...@apache.org.
HBASE-18849 expand "thirdparty" reference to give examples of setting netty location in common testing modules


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

Branch: refs/heads/HBASE-18467
Commit: 58f9cd3586d0d42d26c82fc9b4a9e0c63e44c6d4
Parents: 29a3ff3
Author: Michael Stack <st...@apache.org>
Authored: Tue Sep 19 16:53:21 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Tue Sep 19 16:54:00 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/58f9cd35/src/main/asciidoc/_chapters/developer.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
index 3d010b6..d937d77 100644
--- a/src/main/asciidoc/_chapters/developer.adoc
+++ b/src/main/asciidoc/_chapters/developer.adoc
@@ -2160,6 +2160,23 @@ this property on platforms that favor the bundled .so. See release notes on HBAS
 for more. The complaint you see is something like the following:
 `Cause: java.lang.RuntimeException: Failed construction of Master: class org.apache.hadoop.hbase.master.HMasterorg.apache.hadoop.hbase.shaded.io.netty.channel.epoll.`
 
+If running unit tests and you run into the above message, add the system property
+to your surefire configuration by doing like the below:
+
+[source,xml]
+----
+  <plugin>
+    <artifactId>maven-surefire-plugin</artifactId>
+    <configuration>
+      <systemPropertyVariables>
+        <org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>org.apache.hadoop.hbase.shaded.</org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>
+      </systemPropertyVariables>
+    </configuration>
+  </plugin>
+----
+
+Again the trailing period in the value above is intended.
+
 The `hbase-thirdparty` project has groupid of `org.apache.hbase.thirdparty`.
 As of this writing, it provides three jars; one for netty with an artifactid of
 `hbase-thirdparty-netty`, one for protobuf at `hbase-thirdparty-protobuf` and then


[28/47] hbase git commit: HBASE-18803: Mapreduce job get failed caused by NoClassDefFoundError: org/apache/commons/lang3/ArrayUtils

Posted by bu...@apache.org.
HBASE-18803: Mapreduce job get failed caused by NoClassDefFoundError: org/apache/commons/lang3/ArrayUtils

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


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

Branch: refs/heads/HBASE-18467
Commit: e10d7836ed420754fcf13e3ab7ea1c1c2f1c2098
Parents: 9f0863c
Author: Yi Liang <yl...@us.ibm.com>
Authored: Tue Sep 12 16:49:52 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Mon Sep 18 12:34:55 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e10d7836/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index c3b001e..4dcd048 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -787,7 +787,7 @@ public class TableMapReduceUtil {
       org.apache.hadoop.hbase.ipc.RpcServer.class,                   // hbase-server
       org.apache.hadoop.hbase.CompatibilityFactory.class,            // hbase-hadoop-compat
       org.apache.hadoop.hbase.mapreduce.JobUtil.class,               // hbase-hadoop2-compat
-      org.apache.hadoop.hbase.mapreduce.TableMapper.class,           // hbase-server
+      org.apache.hadoop.hbase.mapreduce.TableMapper.class,           // hbase-mapreduce
       org.apache.hadoop.hbase.metrics.impl.FastLongHistogram.class,  // hbase-metrics
       org.apache.hadoop.hbase.metrics.Snapshot.class,                // hbase-metrics-api
       prefixTreeCodecClass, //  hbase-prefix-tree (if null will be skipped)
@@ -798,7 +798,8 @@ public class TableMapReduceUtil {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations.class,
       org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists.class,
       org.apache.htrace.Trace.class,
-      com.codahale.metrics.MetricRegistry.class);
+      com.codahale.metrics.MetricRegistry.class,
+      org.apache.commons.lang3.ArrayUtils.class);
   }
 
   /**


[07/47] hbase git commit: HBASE-18683 Upgrade hbase to commons-math 3

Posted by bu...@apache.org.
HBASE-18683 Upgrade hbase to commons-math 3

Upgrade commons-math:2.2 to commons-math3:3.6.1
Remove commons-math 2 specific content from LICENSE.vm
Add missing jersey-client dependency to hbase-it module

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


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

Branch: refs/heads/HBASE-18467
Commit: 778f507ada6824e6f388f1500dd56787183b1e23
Parents: c17b491
Author: Peter Somogyi <ps...@cloudera.com>
Authored: Fri Aug 25 13:34:03 2017 +0200
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 14 08:34:28 2017 -0700

----------------------------------------------------------------------
 hbase-it/pom.xml                                |   7 +-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java  |   2 +-
 .../src/main/resources/META-INF/LICENSE.vm      | 310 +++++--------------
 hbase-server/pom.xml                            |   2 +-
 .../hbase/HFilePerformanceEvaluation.java       |   4 +-
 .../compactions/GaussianFileListGenerator.java  |   4 +-
 pom.xml                                         |   4 +-
 7 files changed, 94 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/778f507a/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 5642e3c..6df0b59 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -258,7 +258,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
-      <artifactId>commons-math</artifactId>
+      <artifactId>commons-math3</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
@@ -273,6 +273,11 @@
       <artifactId>javax.ws.rs-api</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-client</artifactId>
+      <version>${jersey.version}</version>
+    </dependency>
     <!-- Hadoop needs Netty 3.x at test scope for the minicluster -->
     <dependency>
       <groupId>io.netty</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/778f507a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
index 97a06f4..71e0d0b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
@@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
+import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/778f507a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
index fcef3bb..e1475a5 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
@@ -896,8 +896,7 @@ facade for Java, which can be obtained at:
     * http://www.slf4j.org/
 #end
 ## Supplemental from commons-math
-## parameter two-and-three true = updated to also cover commons-math 2, just 3 otherwise
-#macro(commons_math_license $two-and-three)
+#macro(commons_math_license)
 ----
 APACHE COMMONS MATH DERIVATIVE WORKS:
 
@@ -910,10 +909,6 @@ are reproduced below.
 For the lmder, lmpar and qrsolv Fortran routine from minpack and translated in
 the LevenbergMarquardtOptimizer class in package
 org.apache.commons.math3.optimization.general
-#if(${two-and-three})
-and package
-org.apache.commons.math.optimization.general
-#end
 Original source copyright and license statement:
 
 Minpack Copyright Notice (1999) University of Chicago.  All rights reserved
@@ -972,9 +967,7 @@ POSSIBILITY OF SUCH LOSS OR DAMAGES.
 Copyright and license statement for the odex Fortran routine developed by
 E. Hairer and G. Wanner and translated in GraggBulirschStoerIntegrator class
 in package org.apache.commons.math3.ode.nonstiff:
-#if(${two-and-three})
-and in package org.apache.commons.math.ode.nonstiff:
-#end
+
 
 Copyright (c) 2004, Ernst Hairer
 
@@ -1002,58 +995,9 @@ NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
 SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 ===============================================================================
 
-Copyright and license statement for the original lapack fortran routines
-translated in EigenDecompositionImpl class in package
-org.apache.commons.math3.linear:
-#if(${two-and-three})
-and package
-org.apache.commons.math.linear:
-#end
-
-Copyright (c) 1992-2008 The University of Tennessee.  All rights reserved.
-
-$COPYRIGHT$
-
-Additional copyrights may follow
-
-$HEADER$
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-- Redistributions of source code must retain the above copyright
-  notice, this list of conditions and the following disclaimer.
-
-- Redistributions in binary form must reproduce the above copyright
-  notice, this list of conditions and the following disclaimer listed
-  in this license in the documentation and/or other materials
-  provided with the distribution.
-
-- Neither the name of the copyright holders nor the names of its
-  contributors may be used to endorse or promote products derived from
-  this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-===============================================================================
-
 Copyright and license statement for the original Mersenne twister C
 routines translated in MersenneTwister class in package
 org.apache.commons.math3.random:
-#if(${two-and-three})
-and package
-org.apache.commons.math.random:
-#end
 
    Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura,
    All rights reserved.
@@ -1087,191 +1031,103 @@ org.apache.commons.math.random:
 
 ===============================================================================
 
-The class "org.apache.commons.math3.exception.util.LocalizedFormatsTest" is
-an adapted version of "OrekitMessagesTest" test class for the Orekit library
-The "org.apache.commons.math3.analysis.interpolation.HermiteInterpolator"
-has been imported from the Orekit space flight dynamics library.
-
-Th Orekit library is described at:
-  https://www.orekit.org/forge/projects/orekit
-The original files are distributed under the terms of the Apache 2 license
-which is: Copyright 2010 CS Communication & Systèmes
-
-#end
-## Supplemental from commons-math if just commons-math 2 is present
-#macro(commons_math_two_license)
-----
-APACHE COMMONS MATH DERIVATIVE WORKS:
-
-The Apache commons-math library includes a number of subcomponents
-whose implementation is derived from original sources written
-in C or Fortran.  License terms of the original sources
-are reproduced below.
+The initial code for shuffling an array (originally in class
+"org.apache.commons.math3.random.RandomDataGenerator", now replaced by
+a method in class "org.apache.commons.math3.util.MathArrays") was
+inspired from the algorithm description provided in
+"Algorithms", by Ian Craw and John Pulham (University of Aberdeen 1999).
+The textbook (containing a proof that the shuffle is uniformly random) is
+available here:
+  http://citeseerx.ist.psu.edu/viewdoc/download;?doi=10.1.1.173.1898&rep=rep1&type=pdf
 
 ===============================================================================
-For the lmder, lmpar and qrsolv Fortran routine from minpack and translated in
-the LevenbergMarquardtOptimizer class in package
-org.apache.commons.math.optimization.general
-Original source copyright and license statement:
+License statement for the direction numbers in the resource files for Sobol sequences.
 
-Minpack Copyright Notice (1999) University of Chicago.  All rights reserved
-
-Redistribution and use in source and binary forms, with or
-without modification, are permitted provided that the
-following conditions are met:
-
-1. Redistributions of source code must retain the above
-copyright notice, this list of conditions and the following
-disclaimer.
+-----------------------------------------------------------------------------
+Licence pertaining to sobol.cc and the accompanying sets of direction numbers
 
-2. Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following
-disclaimer in the documentation and/or other materials
-provided with the distribution.
+-----------------------------------------------------------------------------
+Copyright (c) 2008, Frances Y. Kuo and Stephen Joe
+All rights reserved.
 
-3. The end-user documentation included with the
-redistribution, if any, must include the following
-acknowledgment:
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
 
-   "This product includes software developed by the
-   University of Chicago, as Operator of Argonne National
-   Laboratory.
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
 
-Alternately, this acknowledgment may appear in the software
-itself, if and wherever such third-party acknowledgments
-normally appear.
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
 
-4. WARRANTY DISCLAIMER. THE SOFTWARE IS SUPPLIED "AS IS"
-WITHOUT WARRANTY OF ANY KIND. THE COPYRIGHT HOLDER, THE
-UNITED STATES, THE UNITED STATES DEPARTMENT OF ENERGY, AND
-THEIR EMPLOYEES: (1) DISCLAIM ANY WARRANTIES, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO ANY IMPLIED WARRANTIES
-OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE, TITLE
-OR NON-INFRINGEMENT, (2) DO NOT ASSUME ANY LEGAL LIABILITY
-OR RESPONSIBILITY FOR THE ACCURACY, COMPLETENESS, OR
-USEFULNESS OF THE SOFTWARE, (3) DO NOT REPRESENT THAT USE OF
-THE SOFTWARE WOULD NOT INFRINGE PRIVATELY OWNED RIGHTS, (4)
-DO NOT WARRANT THAT THE SOFTWARE WILL FUNCTION
-UNINTERRUPTED, THAT IT IS ERROR-FREE OR THAT ANY ERRORS WILL
-BE CORRECTED.
+    * Neither the names of the copyright holders nor the names of the
+      University of New South Wales and the University of Waikato
+      and its contributors may be used to endorse or promote products derived
+      from this software without specific prior written permission.
 
-5. LIMITATION OF LIABILITY. IN NO EVENT WILL THE COPYRIGHT
-HOLDER, THE UNITED STATES, THE UNITED STATES DEPARTMENT OF
-ENERGY, OR THEIR EMPLOYEES: BE LIABLE FOR ANY INDIRECT,
-INCIDENTAL, CONSEQUENTIAL, SPECIAL OR PUNITIVE DAMAGES OF
-ANY KIND OR NATURE, INCLUDING BUT NOT LIMITED TO LOSS OF
-PROFITS OR LOSS OF DATA, FOR ANY REASON WHATSOEVER, WHETHER
-SUCH LIABILITY IS ASSERTED ON THE BASIS OF CONTRACT, TORT
-(INCLUDING NEGLIGENCE OR STRICT LIABILITY), OR OTHERWISE,
-EVEN IF ANY OF SAID PARTIES HAS BEEN WARNED OF THE
-POSSIBILITY OF SUCH LOSS OR DAMAGES.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
+EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS BE LIABLE FOR ANY
+DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 ===============================================================================
 
-Copyright and license statement for the odex Fortran routine developed by
-E. Hairer and G. Wanner and translated in GraggBulirschStoerIntegrator class
-in package org.apache.commons.math.ode.nonstiff:
-
-Copyright (c) 2004, Ernst Hairer
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-- Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-
-- Redistributions in binary form must reproduce the above copyright
-notice, this list of conditions and the following disclaimer in the
-documentation and/or other materials provided with the distribution.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
-IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
-TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
-PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR
-CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
-EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
-PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
-PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+The initial commit of package "org.apache.commons.math3.ml.neuralnet" is
+an adapted version of code developed in the context of the Data Processing
+and Analysis Consortium (DPAC) of the "Gaia" project of the European Space
+Agency (ESA).
 ===============================================================================
 
-Copyright and license statement for the original lapack fortran routines
-translated in EigenDecompositionImpl class in package
-org.apache.commons.math.linear:
+The initial commit of the class "org.apache.commons.math3.special.BesselJ" is
+an adapted version of code translated from the netlib Fortran program, rjbesl
+http://www.netlib.org/specfun/rjbesl by R.J. Cody at Argonne National
+Laboratory (USA).  There is no license or copyright statement included with the
+original Fortran sources.
+===============================================================================
 
-Copyright (c) 1992-2008 The University of Tennessee.  All rights reserved.
 
-$COPYRIGHT$
+The BracketFinder (package org.apache.commons.math3.optimization.univariate)
+and PowellOptimizer (package org.apache.commons.math3.optimization.general)
+classes are based on the Python code in module "optimize.py" (version 0.5)
+developed by Travis E. Oliphant for the SciPy library (http://www.scipy.org/)
+Copyright © 2003-2009 SciPy Developers.
 
-Additional copyrights may follow
+SciPy license
+Copyright © 2001, 2002 Enthought, Inc.
+All rights reserved.
 
-$HEADER$
+Copyright © 2003-2013 SciPy Developers.
+All rights reserved.
 
 Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
+modification, are permitted provided that the following conditions are met:
 
-- Redistributions of source code must retain the above copyright
-  notice, this list of conditions and the following disclaimer.
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
 
-- Redistributions in binary form must reproduce the above copyright
-  notice, this list of conditions and the following disclaimer listed
-  in this license in the documentation and/or other materials
-  provided with the distribution.
-
-- Neither the name of the copyright holders nor the names of its
-  contributors may be used to endorse or promote products derived from
-  this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-===============================================================================
-
-Copyright and license statement for the original Mersenne twister C
-routines translated in MersenneTwister class in package
-org.apache.commons.math.random:
-
-   Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura,
-   All rights reserved.
-
-   Redistribution and use in source and binary forms, with or without
-   modification, are permitted provided that the following conditions
-   are met:
-
-     1. Redistributions of source code must retain the above copyright
-        notice, this list of conditions and the following disclaimer.
-
-     2. Redistributions in binary form must reproduce the above copyright
-        notice, this list of conditions and the following disclaimer in the
-        documentation and/or other materials provided with the distribution.
-
-     3. The names of its contributors may not be used to endorse or promote
-        products derived from this software without specific prior written
-        permission.
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
 
-   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-   A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER OR
-   CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
-   EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
-   PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
-   PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-   LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-   NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-   SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+    * Neither the name of Enthought nor the names of the SciPy Developers may
+      be used to endorse or promote products derived from this software without
+      specific prior written permission.
 
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY
+EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE FOR ANY
+DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 ===============================================================================
 #end
 ## Hadoop supplemental
@@ -1472,8 +1328,7 @@ You can redistribute it and/or modify it under either the terms of the
 ## gather up EPL 1.0 works
 #set($epl = [])
 ## track commons-math
-#set($commons-math-two = false)
-#set($commons-math-three = false)
+#set($commons-math = false)
 ## track if we need jruby additionals.
 #set($jruby = false)
 ## track hadoops
@@ -1518,11 +1373,8 @@ ${dep.scm.url}
 #end
 #set($aggregated=false)
 ## Check for our set of known dependencies that require manual LICENSE additions.
-#if($dep.artifactId.equals("commons-math"))
-#set($commons-math-two=true)
-#end
 #if($dep.artifactId.equals("commons-math3"))
-#set($commons-math-three=true)
+#set($commons-math=true)
 #end
 #if($dep.artifactId.equals("jruby-complete"))
 #set($jruby=true)
@@ -1625,10 +1477,8 @@ ${dep.scm.url}
 #if($hadoop)
 #hadoop_license()
 #end
-#if($commons-math-three)
-#commons_math_license($commons-math-two)
-#elseif($commons-math-two)
-#commons_math_two_license()
+#if($commons-math)
+#commons_math_license()
 #end
 #if(!(${mit.isEmpty()}))
 ====

http://git-wip-us.apache.org/repos/asf/hbase/blob/778f507a/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index c1cb3a0..1d888d3 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -545,7 +545,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
-      <artifactId>commons-math</artifactId>
+      <artifactId>commons-math3</artifactId>
     </dependency>
     <dependency>
       <groupId>log4j</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/778f507a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
index 02ef695..b50a2b8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
@@ -24,8 +24,8 @@ import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math.random.RandomData;
-import org.apache.commons.math.random.RandomDataImpl;
+import org.apache.commons.math3.random.RandomData;
+import org.apache.commons.math3.random.RandomDataImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

http://git-wip-us.apache.org/repos/asf/hbase/blob/778f507a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
index fb8c30a..a2964ff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
@@ -22,8 +22,8 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.commons.math.random.GaussianRandomGenerator;
-import org.apache.commons.math.random.MersenneTwister;
+import org.apache.commons.math3.random.GaussianRandomGenerator;
+import org.apache.commons.math3.random.MersenneTwister;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 
 class GaussianFileListGenerator extends StoreFileListGenerator {

http://git-wip-us.apache.org/repos/asf/hbase/blob/778f507a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 69a1892..29a2505 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1391,7 +1391,7 @@
     <commons-io.version>2.5</commons-io.version>
     <commons-lang3.version>3.6</commons-lang3.version>
     <commons-logging.version>1.2</commons-logging.version>
-    <commons-math.version>2.2</commons-math.version>
+    <commons-math.version>3.6.1</commons-math.version>
     <disruptor.version>3.3.6</disruptor.version>
     <!-- Do not use versions earlier than 3.2.2 due to a security vulnerability -->
     <collections.version>4.1</collections.version>
@@ -1812,7 +1812,7 @@
       </dependency>
       <dependency>
         <groupId>org.apache.commons</groupId>
-        <artifactId>commons-math</artifactId>
+        <artifactId>commons-math3</artifactId>
         <version>${commons-math.version}</version>
       </dependency>
       <dependency>


[04/47] hbase git commit: HBASE-18453 CompactionRequest should not be exposed to user directly

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 6edf006..86a24ad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -17,6 +17,59 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
+import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.lang.reflect.Constructor;
+import java.nio.ByteBuffer;
+import java.text.ParseException;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Optional;
+import java.util.RandomAccess;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -90,6 +143,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.Write
 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@@ -97,7 +151,6 @@ import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@@ -143,58 +196,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
-import java.io.EOFException;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.lang.reflect.Constructor;
-import java.nio.ByteBuffer;
-import java.text.ParseException;
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.RandomAccess;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.LongAdder;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Function;
-
-import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
-import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
-
 @SuppressWarnings("deprecation")
 @InterfaceAudience.Private
 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
@@ -254,9 +255,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   // - the thread that owns the lock (allow reentrancy)
   // - reference count of (reentrant) locks held by the thread
   // - the row itself
-  private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows =
+      new ConcurrentHashMap<>();
 
-  protected final Map<byte[], Store> stores = new ConcurrentSkipListMap<>(Bytes.BYTES_RAWCOMPARATOR);
+  protected final Map<byte[], HStore> stores =
+      new ConcurrentSkipListMap<>(Bytes.BYTES_RAWCOMPARATOR);
 
   // TODO: account for each registered handler in HeapSize computation
   private Map<String, com.google.protobuf.Service> coprocessorServiceHandlers = Maps.newHashMap();
@@ -513,7 +516,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   /** A result object from prepare flush cache stage */
   @VisibleForTesting
   static class PrepareFlushResult {
-    final FlushResult result; // indicating a failure result from prepare
+    final FlushResultImpl result; // indicating a failure result from prepare
     final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
     final TreeMap<byte[], List<Path>> committedFiles;
     final TreeMap<byte[], MemstoreSize> storeFlushableSize;
@@ -523,7 +526,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     final MemstoreSize totalFlushableSize;
 
     /** Constructs an early exit case */
-    PrepareFlushResult(FlushResult result, long flushSeqId) {
+    PrepareFlushResult(FlushResultImpl result, long flushSeqId) {
       this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, new MemstoreSize());
     }
 
@@ -538,7 +541,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
 
     private PrepareFlushResult(
-      FlushResult result,
+        FlushResultImpl result,
       TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
       TreeMap<byte[], List<Path>> committedFiles,
       TreeMap<byte[], MemstoreSize> storeFlushableSize, long startTime, long flushSeqId,
@@ -616,7 +619,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   final long rowProcessorTimeout;
 
   // Last flush time for each Store. Useful when we are flushing for each column
-  private final ConcurrentMap<Store, Long> lastStoreFlushTimeMap = new ConcurrentHashMap<>();
+  private final ConcurrentMap<HStore, Long> lastStoreFlushTimeMap = new ConcurrentHashMap<>();
 
   final RegionServerServices rsServices;
   private RegionServerAccounting rsAccounting;
@@ -802,7 +805,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     this.disallowWritesInRecovering =
         conf.getBoolean(HConstants.DISALLOW_WRITES_IN_RECOVERING,
           HConstants.DEFAULT_DISALLOW_WRITES_IN_RECOVERING_CONFIG);
-    configurationManager = Optional.absent();
+    configurationManager = Optional.empty();
 
     // disable stats tracking system tables, but check the config for everything else
     this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals(
@@ -902,22 +905,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     long maxSeqId = initializeStores(reporter, status);
     this.mvcc.advanceTo(maxSeqId);
     if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
-      List<Store> stores = this.getStores();  // update the stores that we are replaying
+      Collection<HStore> stores = this.stores.values();
       try {
-        for (Store store : stores) {
-          ((HStore) store).startReplayingFromWAL();
-        }
+        // update the stores that we are replaying
+        stores.forEach(HStore::startReplayingFromWAL);
         // Recover any edits if available.
         maxSeqId = Math.max(maxSeqId,
-            replayRecoveredEditsIfAny(this.fs.getRegionDir(), maxSeqIdInStores, reporter, status));
+          replayRecoveredEditsIfAny(this.fs.getRegionDir(), maxSeqIdInStores, reporter, status));
         // Make sure mvcc is up to max.
         this.mvcc.advanceTo(maxSeqId);
       } finally {
-        for (Store store : stores) {            // update the stores that we are done replaying
-          ((HStore)store).stopReplayingFromWAL();
-        }
+        // update the stores that we are done replaying
+        stores.forEach(HStore::startReplayingFromWAL);
       }
-
     }
     this.lastReplayedOpenRegionSeqId = maxSeqId;
 
@@ -947,7 +947,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     this.flushPolicy = FlushPolicyFactory.create(this, conf);
 
     long lastFlushTime = EnvironmentEdgeManager.currentTime();
-    for (Store store: stores.values()) {
+    for (HStore store: stores.values()) {
       this.lastStoreFlushTimeMap.put(store, lastFlushTime);
     }
 
@@ -988,10 +988,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @return Highest sequenceId found out in a Store.
    * @throws IOException
    */
-  private long initializeStores(final CancelableProgressable reporter, MonitoredTask status)
-  throws IOException {
+  private long initializeStores(CancelableProgressable reporter, MonitoredTask status)
+      throws IOException {
     // Load in all the HStores.
-
     long maxSeqId = -1;
     // initialized to -1 so that we pick up MemstoreTS from column families
     long maxMemstoreTS = -1;
@@ -1050,11 +1049,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (!allStoresOpened) {
           // something went wrong, close all opened stores
           LOG.error("Could not initialize all stores for the region=" + this);
-          for (Store store : this.stores.values()) {
+          for (HStore store : this.stores.values()) {
             try {
               store.close();
             } catch (IOException e) {
-              LOG.warn(e.getMessage());
+              LOG.warn("close store failed", e);
             }
           }
         }
@@ -1079,11 +1078,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   private NavigableMap<byte[], List<Path>> getStoreFiles() {
     NavigableMap<byte[], List<Path>> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for (Store store: getStores()) {
+    for (HStore store : stores.values()) {
       Collection<StoreFile> storeFiles = store.getStorefiles();
-      if (storeFiles == null) continue;
+      if (storeFiles == null) {
+        continue;
+      }
       List<Path> storeFileNames = new ArrayList<>();
-      for (StoreFile storeFile: storeFiles) {
+      for (StoreFile storeFile : storeFiles) {
         storeFileNames.add(storeFile.getPath());
       }
       allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames);
@@ -1121,10 +1122,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @return True if this region has references.
    */
   public boolean hasReferences() {
-    for (Store store : this.stores.values()) {
-      if (store.hasReferences()) return true;
-    }
-    return false;
+    return stores.values().stream().anyMatch(HStore::hasReferences);
   }
 
   public void blockUpdates() {
@@ -1137,19 +1135,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @Override
   public HDFSBlocksDistribution getHDFSBlocksDistribution() {
-    HDFSBlocksDistribution hdfsBlocksDistribution =
-      new HDFSBlocksDistribution();
-    synchronized (this.stores) {
-      for (Store store : this.stores.values()) {
-        Collection<StoreFile> storeFiles = store.getStorefiles();
-        if (storeFiles == null) continue;
-        for (StoreFile sf : storeFiles) {
-          HDFSBlocksDistribution storeFileBlocksDistribution =
-            sf.getHDFSBlockDistribution();
-          hdfsBlocksDistribution.add(storeFileBlocksDistribution);
-        }
-      }
-    }
+    HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
+    stores.values().stream().filter(s -> s.getStorefiles() != null)
+        .flatMap(s -> s.getStorefiles().stream()).map(StoreFile::getHDFSBlockDistribution)
+        .forEachOrdered(hdfsBlocksDistribution::add);
     return hdfsBlocksDistribution;
   }
 
@@ -1161,8 +1150,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @return The HDFS blocks distribution for the given region.
    * @throws IOException
    */
-  public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
-      final TableDescriptor tableDescriptor, final HRegionInfo regionInfo) throws IOException {
+  public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
+      TableDescriptor tableDescriptor, HRegionInfo regionInfo) throws IOException {
     Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
     return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
   }
@@ -1176,9 +1165,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @return The HDFS blocks distribution for the given region.
    * @throws IOException
    */
-  public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
-      final TableDescriptor tableDescriptor, final HRegionInfo regionInfo,  Path tablePath)
-      throws IOException {
+  public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
+      TableDescriptor tableDescriptor, HRegionInfo regionInfo, Path tablePath) throws IOException {
     HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
     FileSystem fs = tablePath.getFileSystem(conf);
 
@@ -1407,9 +1395,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       new Throwable("LOGGING: REMOVE"));
     // REMOVE BELOW!!!!
     LOG.info("DEBUG LIST ALL FILES");
-    for (Store store: this.stores.values()) {
+    for (HStore store : this.stores.values()) {
       LOG.info("store " + store.getColumnFamilyName());
-      for (StoreFile sf: store.getStorefiles()) {
+      for (StoreFile sf : store.getStorefiles()) {
         LOG.info(sf.toStringDetailed());
       }
     }
@@ -1667,7 +1655,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           new ExecutorCompletionService<>(storeCloserThreadPool);
 
         // close each store in parallel
-        for (final Store store : stores.values()) {
+        for (HStore store : stores.values()) {
           MemstoreSize flushableSize = store.getSizeToFlush();
           if (!(abort || flushableSize.getDataSize() == 0 || writestate.readOnly)) {
             if (getRegionServerServices() != null) {
@@ -1740,11 +1728,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   private long getMemstoreHeapSize() {
-    long size = 0;
-    for (Store s : this.stores.values()) {
-      size += s.getSizeOfMemStore().getHeapSize();
-    }
-    return size;
+    return stores.values().stream().mapToLong(s -> s.getSizeOfMemStore().getHeapSize()).sum();
   }
 
   @Override
@@ -1902,17 +1886,25 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
     long result = Long.MAX_VALUE;
-    for (Store store : getStores()) {
+    for (HStore store : stores.values()) {
       Collection<StoreFile> storeFiles = store.getStorefiles();
-      if (storeFiles == null) continue;
+      if (storeFiles == null) {
+        continue;
+      }
       for (StoreFile file : storeFiles) {
         StoreFileReader sfReader = file.getReader();
-        if (sfReader == null) continue;
+        if (sfReader == null) {
+          continue;
+        }
         HFile.Reader reader = sfReader.getHFileReader();
-        if (reader == null) continue;
+        if (reader == null) {
+          continue;
+        }
         if (majorCompactionOnly) {
           byte[] val = reader.loadFileInfo().get(StoreFile.MAJOR_COMPACTION_KEY);
-          if (val == null || !Bytes.toBoolean(val)) continue;
+          if (val == null || !Bytes.toBoolean(val)) {
+            continue;
+          }
         }
         result = Math.min(result, reader.getFileContext().getFileCreateTime());
       }
@@ -1942,20 +1934,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   // These methods are meant to be called periodically by the HRegionServer for
   // upkeep.
   //////////////////////////////////////////////////////////////////////////////
-
-  /** @return returns size of largest HStore. */
+  /**
+   * @return returns size of largest HStore.
+   */
   public long getLargestHStoreSize() {
-    long size = 0;
-    for (Store h : stores.values()) {
-      long storeSize = h.getSize();
-      if (storeSize > size) {
-        size = storeSize;
-      }
-    }
-    return size;
+    return stores.values().stream().mapToLong(HStore::getSize).max().orElse(0L);
   }
 
-  /*
+  /**
    * Do preparation for pending compaction.
    * @throws IOException
    */
@@ -1964,19 +1950,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @Override
   public void triggerMajorCompaction() throws IOException {
-    for (Store s : getStores()) {
-      s.triggerMajorCompaction();
-    }
+    stores.values().forEach(HStore::triggerMajorCompaction);
   }
 
   @Override
-  public void compact(final boolean majorCompaction) throws IOException {
+  public void compact(boolean majorCompaction) throws IOException {
     if (majorCompaction) {
       triggerMajorCompaction();
     }
-    for (Store s : getStores()) {
-      CompactionContext compaction = s.requestCompaction();
-      if (compaction != null) {
+    for (HStore s : stores.values()) {
+      Optional<CompactionContext> compaction = s.requestCompaction();
+      if (compaction.isPresent()) {
         ThroughputController controller = null;
         if (rsServices != null) {
           controller = CompactionThroughputControllerFactory.create(rsServices, conf);
@@ -1984,43 +1968,41 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (controller == null) {
           controller = NoLimitThroughputController.INSTANCE;
         }
-        compact(compaction, s, controller, null);
+        compact(compaction.get(), s, controller, null);
       }
     }
   }
 
   /**
-   * This is a helper function that compact all the stores synchronously
+   * This is a helper function that compact all the stores synchronously.
+   * <p>
    * It is used by utilities and testing
-   *
-   * @throws IOException e
    */
+  @VisibleForTesting
   public void compactStores() throws IOException {
-    for (Store s : getStores()) {
-      CompactionContext compaction = s.requestCompaction();
-      if (compaction != null) {
-        compact(compaction, s, NoLimitThroughputController.INSTANCE, null);
+    for (HStore s : stores.values()) {
+      Optional<CompactionContext> compaction = s.requestCompaction();
+      if (compaction.isPresent()) {
+        compact(compaction.get(), s, NoLimitThroughputController.INSTANCE, null);
       }
     }
   }
 
   /**
-   * This is a helper function that compact the given store
+   * This is a helper function that compact the given store.
+   * <p>
    * It is used by utilities and testing
-   *
-   * @throws IOException e
    */
   @VisibleForTesting
-  void compactStore(byte[] family, ThroughputController throughputController)
-      throws IOException {
-    Store s = getStore(family);
-    CompactionContext compaction = s.requestCompaction();
-    if (compaction != null) {
-      compact(compaction, s, throughputController, null);
+  void compactStore(byte[] family, ThroughputController throughputController) throws IOException {
+    HStore s = getStore(family);
+    Optional<CompactionContext> compaction = s.requestCompaction();
+    if (compaction.isPresent()) {
+      compact(compaction.get(), s, throughputController, null);
     }
   }
 
-  /*
+  /**
    * Called by compaction thread and after region is opened to compact the
    * HStores if necessary.
    *
@@ -2035,12 +2017,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param throughputController
    * @return whether the compaction completed
    */
-  public boolean compact(CompactionContext compaction, Store store,
+  public boolean compact(CompactionContext compaction, HStore store,
       ThroughputController throughputController) throws IOException {
     return compact(compaction, store, throughputController, null);
   }
 
-  public boolean compact(CompactionContext compaction, Store store,
+  public boolean compact(CompactionContext compaction, HStore store,
       ThroughputController throughputController, User user) throws IOException {
     assert compaction != null && compaction.hasSelection();
     assert !compaction.getRequest().getFiles().isEmpty();
@@ -2214,7 +2196,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * because a Snapshot was not properly persisted. The region is put in closing mode, and the
    * caller MUST abort after this.
    */
-  public FlushResult flushcache(boolean forceFlushAllStores, boolean writeFlushRequestWalMarker)
+  public FlushResultImpl flushcache(boolean forceFlushAllStores, boolean writeFlushRequestWalMarker)
       throws IOException {
     // fail-fast instead of waiting on the lock
     if (this.closing.get()) {
@@ -2261,10 +2243,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
 
       try {
-        Collection<Store> specificStoresToFlush =
+        Collection<HStore> specificStoresToFlush =
             forceFlushAllStores ? stores.values() : flushPolicy.selectStoresToFlush();
-        FlushResult fs = internalFlushcache(specificStoresToFlush,
-          status, writeFlushRequestWalMarker);
+        FlushResultImpl fs =
+            internalFlushcache(specificStoresToFlush, status, writeFlushRequestWalMarker);
 
         if (coprocessorHost != null) {
           status.setStatus("Running post-flush coprocessor hooks");
@@ -2297,7 +2279,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * that you always flush all stores). Otherwise the method will always
    * returns true which will make a lot of flush requests.
    */
-  boolean shouldFlushStore(Store store) {
+  boolean shouldFlushStore(HStore store) {
     long earliest = this.wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
       store.getColumnFamilyDescriptor().getName()) - 1;
     if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
@@ -2349,7 +2331,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
     //since we didn't flush in the recent past, flush now if certain conditions
     //are met. Return true on first such memstore hit.
-    for (Store s : getStores()) {
+    for (Store s : stores.values()) {
       if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
         // we have an old enough edit in the memstore, flush
         whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
@@ -2361,39 +2343,33 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   /**
    * Flushing all stores.
-   *
    * @see #internalFlushcache(Collection, MonitoredTask, boolean)
    */
-  private FlushResult internalFlushcache(MonitoredTask status)
-      throws IOException {
+  private FlushResult internalFlushcache(MonitoredTask status) throws IOException {
     return internalFlushcache(stores.values(), status, false);
   }
 
   /**
    * Flushing given stores.
-   *
    * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean)
    */
-  private FlushResult internalFlushcache(final Collection<Store> storesToFlush,
-      MonitoredTask status, boolean writeFlushWalMarker) throws IOException {
-    return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush,
-        status, writeFlushWalMarker);
+  private FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status,
+      boolean writeFlushWalMarker) throws IOException {
+    return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush, status,
+      writeFlushWalMarker);
   }
 
   /**
-   * Flush the memstore. Flushing the memstore is a little tricky. We have a lot
-   * of updates in the memstore, all of which have also been written to the wal.
-   * We need to write those updates in the memstore out to disk, while being
-   * able to process reads/writes as much as possible during the flush
-   * operation.
+   * Flush the memstore. Flushing the memstore is a little tricky. We have a lot of updates in the
+   * memstore, all of which have also been written to the wal. We need to write those updates in the
+   * memstore out to disk, while being able to process reads/writes as much as possible during the
+   * flush operation.
    * <p>
-   * This method may block for some time. Every time you call it, we up the
-   * regions sequence id even if we don't flush; i.e. the returned region id
-   * will be at least one larger than the last edit applied to this region. The
-   * returned id does not refer to an actual edit. The returned id can be used
-   * for say installing a bulk loaded file just ahead of the last hfile that was
-   * the result of this flush, etc.
-   *
+   * This method may block for some time. Every time you call it, we up the regions sequence id even
+   * if we don't flush; i.e. the returned region id will be at least one larger than the last edit
+   * applied to this region. The returned id does not refer to an actual edit. The returned id can
+   * be used for say installing a bulk loaded file just ahead of the last hfile that was the result
+   * of this flush, etc.
    * @param wal Null if we're NOT to go via wal.
    * @param myseqid The seqid to use if <code>wal</code> is null writing out flush file.
    * @param storesToFlush The list of stores to flush.
@@ -2401,9 +2377,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @throws IOException general io exceptions
    * @throws DroppedSnapshotException Thrown when replay of WAL is required.
    */
-  protected FlushResult internalFlushcache(final WAL wal, final long myseqid,
-      final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
-          throws IOException {
+  protected FlushResultImpl internalFlushcache(WAL wal, long myseqid, Collection<HStore> storesToFlush,
+      MonitoredTask status, boolean writeFlushWalMarker) throws IOException {
     PrepareFlushResult result
       = internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker);
     if (result.result == null) {
@@ -2415,9 +2390,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
       justification="FindBugs seems confused about trxId")
-  protected PrepareFlushResult internalPrepareFlushCache(final WAL wal, final long myseqid,
-      final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
-  throws IOException {
+  protected PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid,
+      Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
+      throws IOException {
     if (this.rsServices != null && this.rsServices.isAborted()) {
       // Don't flush when server aborting, it's unsafe
       throw new IOException("Aborting flush because server is aborted...");
@@ -2439,11 +2414,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
           // sure just beyond the last appended region edit and not associated with any edit
           // (useful as marker when bulk loading, etc.).
-          FlushResult flushResult = null;
           if (wal != null) {
             writeEntry = mvcc.begin();
             long flushOpSeqId = writeEntry.getWriteNumber();
-            flushResult = new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
+            FlushResultImpl flushResult = new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
               flushOpSeqId, "Nothing to flush",
             writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
             mvcc.completeAndWait(writeEntry);
@@ -2479,9 +2453,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     MemstoreSize totalSizeOfFlushableStores = new MemstoreSize();
 
     Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>();
-    for (Store store: storesToFlush) {
+    for (HStore store : storesToFlush) {
       flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(),
-          ((HStore) store).preFlushSeqIDEstimation());
+        store.preFlushSeqIDEstimation());
     }
 
     TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
@@ -2517,7 +2491,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         flushedSeqId = flushOpSeqId = myseqid;
       }
 
-      for (Store s : storesToFlush) {
+      for (HStore s : storesToFlush) {
         MemstoreSize flushableSize = s.getSizeToFlush();
         totalSizeOfFlushableStores.incMemstoreSize(flushableSize);
         storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), s.createFlushContext(flushOpSeqId));
@@ -2555,7 +2529,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   /**
    * Utility method broken out of internalPrepareFlushCache so that method is smaller.
    */
-  private void logFatLineOnFlush(final Collection<Store> storesToFlush, final long sequenceId) {
+  private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) {
     if (!LOG.isInfoEnabled()) {
       return;
     }
@@ -2563,7 +2537,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     StringBuilder perCfExtras = null;
     if (!isAllFamilies(storesToFlush)) {
       perCfExtras = new StringBuilder();
-      for (Store store: storesToFlush) {
+      for (HStore store: storesToFlush) {
         perCfExtras.append("; ").append(store.getColumnFamilyName());
         perCfExtras.append("=")
             .append(StringUtils.byteDesc(store.getSizeToFlush().getDataSize()));
@@ -2611,7 +2585,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   /**
    * @return True if passed Set is all families in the region.
    */
-  private boolean isAllFamilies(final Collection<Store> families) {
+  private boolean isAllFamilies(Collection<HStore> families) {
     return families == null || this.stores.size() == families.size();
   }
 
@@ -2639,11 +2613,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
       justification="Intentional; notify is about completed flush")
-  protected FlushResult internalFlushCacheAndCommit(
-        final WAL wal, MonitoredTask status, final PrepareFlushResult prepareResult,
-        final Collection<Store> storesToFlush)
-    throws IOException {
-
+  protected FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status,
+      PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException {
     // prepare flush context is carried via PrepareFlushResult
     TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs;
     TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles;
@@ -2673,7 +2644,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
       // Switch snapshot (in memstore) -> new hfile (thus causing
       // all the store scanners to reset/reseek).
-      Iterator<Store> it = storesToFlush.iterator();
+      Iterator<HStore> it = storesToFlush.iterator();
       // stores.values() and storeFlushCtxs have same order
       for (StoreFlushContext flush : storeFlushCtxs.values()) {
         boolean needsCompaction = flush.commit(status);
@@ -2746,7 +2717,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
 
     // Record latest flush time
-    for (Store store: storesToFlush) {
+    for (HStore store: storesToFlush) {
       this.lastStoreFlushTimeMap.put(store, startTime);
     }
 
@@ -4002,34 +3973,34 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  /*
+  /**
    * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be
-   *  set; when set we will run operations that make sense in the increment/append scenario but
-   *  that do not make sense otherwise.
-   * @see #applyToMemstore(Store, Cell, long)
+   *          set; when set we will run operations that make sense in the increment/append scenario
+   *          but that do not make sense otherwise.
+   * @see #applyToMemstore(HStore, Cell, long)
    */
-  private void applyToMemstore(final Store store, final List<Cell> cells, final boolean delta,
+  private void applyToMemstore(HStore store, List<Cell> cells, boolean delta,
       MemstoreSize memstoreSize) throws IOException {
     // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!!
     boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
     if (upsert) {
-      ((HStore) store).upsert(cells, getSmallestReadPoint(), memstoreSize);
+      store.upsert(cells, getSmallestReadPoint(), memstoreSize);
     } else {
-      ((HStore) store).add(cells, memstoreSize);
+      store.add(cells, memstoreSize);
     }
   }
 
-  /*
-   * @see #applyToMemstore(Store, List, boolean, boolean, long)
+  /**
+   * @see #applyToMemstore(HStore, List, boolean, boolean, long)
    */
-  private void applyToMemstore(final Store store, final Cell cell, MemstoreSize memstoreSize)
-  throws IOException {
+  private void applyToMemstore(HStore store, Cell cell, MemstoreSize memstoreSize)
+      throws IOException {
     // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!!
     if (store == null) {
       checkFamily(CellUtil.cloneFamily(cell));
       // Unreachable because checkFamily will throw exception
     }
-    ((HStore) store).add(cell, memstoreSize);
+    store.add(cell, memstoreSize);
   }
 
   @Override
@@ -4368,7 +4339,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             // Figure which store the edit is meant for.
             if (store == null || !CellUtil.matchingFamily(cell,
                 store.getColumnFamilyDescriptor().getName())) {
-              store = getHStore(cell);
+              store = getStore(cell);
             }
             if (store == null) {
               // This should never happen.  Perhaps schema was changed between
@@ -4497,7 +4468,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
       startRegionOperation(Operation.REPLAY_EVENT);
       try {
-        HStore store = this.getHStore(compaction.getFamilyName().toByteArray());
+        HStore store = this.getStore(compaction.getFamilyName().toByteArray());
         if (store == null) {
           LOG.warn(getRegionInfo().getEncodedName() + " : "
               + "Found Compaction WAL edit for deleted family:"
@@ -4567,10 +4538,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
     long flushSeqId = flush.getFlushSequenceNumber();
 
-    HashSet<Store> storesToFlush = new HashSet<>();
+    HashSet<HStore> storesToFlush = new HashSet<>();
     for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
       byte[] family = storeFlush.getFamilyName().toByteArray();
-      Store store = getStore(family);
+      HStore store = getStore(family);
       if (store == null) {
         LOG.warn(getRegionInfo().getEncodedName() + " : "
           + "Received a flush start marker from primary, but the family is not found. Ignoring"
@@ -4807,7 +4778,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       throws IOException {
     for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
       byte[] family = storeFlush.getFamilyName().toByteArray();
-      Store store = getStore(family);
+      HStore store = getStore(family);
       if (store == null) {
         LOG.warn(getRegionInfo().getEncodedName() + " : "
             + "Received a flush commit marker from primary, but the family is not found."
@@ -4843,7 +4814,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * if the memstore edits have seqNums smaller than the given seq id
    * @throws IOException
    */
-  private MemstoreSize dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException {
+  private MemstoreSize dropMemstoreContentsForSeqId(long seqId, HStore store) throws IOException {
     MemstoreSize totalFreedSize = new MemstoreSize();
     this.updatesLock.writeLock().lock();
     try {
@@ -4857,7 +4828,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
         // Prepare flush (take a snapshot) and then abort (drop the snapshot)
         if (store == null) {
-          for (Store s : stores.values()) {
+          for (HStore s : stores.values()) {
             totalFreedSize.incMemstoreSize(doDropStoreMemstoreContentsForSeqId(s, currentSeqId));
           }
         } else {
@@ -4874,7 +4845,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return totalFreedSize;
   }
 
-  private MemstoreSize doDropStoreMemstoreContentsForSeqId(Store s, long currentSeqId)
+  private MemstoreSize doDropStoreMemstoreContentsForSeqId(HStore s, long currentSeqId)
       throws IOException {
     MemstoreSize flushableSize = s.getSizeToFlush();
     this.decrMemstoreSize(flushableSize);
@@ -4965,7 +4936,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
           // stores of primary may be different now
           byte[] family = storeDescriptor.getFamilyName().toByteArray();
-          Store store = getStore(family);
+          HStore store = getStore(family);
           if (store == null) {
             LOG.warn(getRegionInfo().getEncodedName() + " : "
                 + "Received a region open marker from primary, but the family is not found. "
@@ -5081,7 +5052,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
           // stores of primary may be different now
           family = storeDescriptor.getFamilyName().toByteArray();
-          HStore store = getHStore(family);
+          HStore store = getStore(family);
           if (store == null) {
             LOG.warn(getRegionInfo().getEncodedName() + " : "
                     + "Received a bulk load marker from primary, but the family is not found. "
@@ -5119,9 +5090,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     if (writestate.flushing) {
       boolean canDrop = true;
       if (prepareFlushResult.storeFlushCtxs != null) {
-        for (Entry<byte[], StoreFlushContext> entry
-            : prepareFlushResult.storeFlushCtxs.entrySet()) {
-          Store store = getStore(entry.getKey());
+        for (Entry<byte[], StoreFlushContext> entry : prepareFlushResult.storeFlushCtxs
+            .entrySet()) {
+          HStore store = getStore(entry.getKey());
           if (store == null) {
             continue;
           }
@@ -5164,9 +5135,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     startRegionOperation(); // obtain region close lock
     try {
-      Map<Store, Long> map = new HashMap<>();
+      Map<HStore, Long> map = new HashMap<>();
       synchronized (writestate) {
-        for (Store store : getStores()) {
+        for (HStore store : stores.values()) {
           // TODO: some stores might see new data from flush, while others do not which
           // MIGHT break atomic edits across column families.
           long maxSeqIdBefore = store.getMaxSequenceId();
@@ -5207,10 +5178,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         dropPrepareFlushIfPossible();
 
         // advance the mvcc read point so that the new flushed files are visible.
-          // either greater than flush seq number or they were already picked up via flush.
-          for (Store s : getStores()) {
-            mvcc.advanceTo(s.getMaxMemstoreTS());
-          }
+        // either greater than flush seq number or they were already picked up via flush.
+        for (HStore s : stores.values()) {
+          mvcc.advanceTo(s.getMaxMemstoreTS());
+        }
 
 
         // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely
@@ -5222,7 +5193,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
       }
       if (!map.isEmpty()) {
-        for (Map.Entry<Store, Long> entry : map.entrySet()) {
+        for (Map.Entry<HStore, Long> entry : map.entrySet()) {
           // Drop the memstore contents if they are now smaller than the latest seen flushed file
           totalFreedDataSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey())
               .getDataSize();
@@ -5242,13 +5213,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private void logRegionFiles() {
     if (LOG.isTraceEnabled()) {
       LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
-      for (Store s : stores.values()) {
-        Collection<StoreFile> storeFiles = s.getStorefiles();
-        if (storeFiles == null) continue;
-        for (StoreFile sf : storeFiles) {
-          LOG.trace(getRegionInfo().getEncodedName() + " : " + sf);
-        }
-      }
+      stores.values().stream().filter(s -> s.getStorefiles() != null)
+          .flatMap(s -> s.getStorefiles().stream())
+          .forEachOrdered(sf -> LOG.trace(getRegionInfo().getEncodedName() + " : " + sf));
     }
   }
 
@@ -5272,17 +5239,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       + " does not match this region: " + this.getRegionInfo());
   }
 
-  /*
+  /**
    * Used by tests
    * @param s Store to add edit too.
    * @param cell Cell to add.
    * @param memstoreSize
    */
-  protected void restoreEdit(final HStore s, final Cell cell, MemstoreSize memstoreSize) {
+  @VisibleForTesting
+  protected void restoreEdit(HStore s, Cell cell, MemstoreSize memstoreSize) {
     s.add(cell, memstoreSize);
   }
 
-  /*
+  /**
    * @param fs
    * @param p File to check.
    * @return True if file was zero-length (and if so, we'll delete it in here).
@@ -5291,7 +5259,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
       throws IOException {
     FileStatus stat = fs.getFileStatus(p);
-    if (stat.getLen() > 0) return false;
+    if (stat.getLen() > 0) {
+      return false;
+    }
     LOG.warn("File " + p + " is zero-length, deleting.");
     fs.delete(p, false);
     return true;
@@ -5311,49 +5281,39 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   @Override
-  public Store getStore(final byte[] column) {
-    return getHStore(column);
-  }
-
-  public HStore getHStore(final byte[] column) {
-    return (HStore) this.stores.get(column);
+  public HStore getStore(byte[] column) {
+    return this.stores.get(column);
   }
 
   /**
-   * Return HStore instance. Does not do any copy: as the number of store is limited, we
-   *  iterate on the list.
+   * Return HStore instance. Does not do any copy: as the number of store is limited, we iterate on
+   * the list.
    */
-  private HStore getHStore(Cell cell) {
-    for (Map.Entry<byte[], Store> famStore : stores.entrySet()) {
-      if (CellUtil.matchingFamily(cell, famStore.getKey(), 0, famStore.getKey().length)) {
-        return (HStore) famStore.getValue();
-      }
-    }
-
-    return null;
+  private HStore getStore(Cell cell) {
+    return stores.entrySet().stream().filter(e -> CellUtil.matchingFamily(cell, e.getKey()))
+        .map(e -> e.getValue()).findFirst().orElse(null);
   }
 
   @Override
-  public List<Store> getStores() {
-    List<Store> list = new ArrayList<>(stores.size());
-    list.addAll(stores.values());
-    return list;
+  public List<HStore> getStores() {
+    return new ArrayList<>(stores.values());
   }
 
   @Override
-  public List<String> getStoreFileList(final byte [][] columns)
-    throws IllegalArgumentException {
+  public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException {
     List<String> storeFileNames = new ArrayList<>();
-    synchronized(closeLock) {
-      for(byte[] column : columns) {
-        Store store = this.stores.get(column);
+    synchronized (closeLock) {
+      for (byte[] column : columns) {
+        HStore store = this.stores.get(column);
         if (store == null) {
-          throw new IllegalArgumentException("No column family : " +
-              new String(column) + " available");
+          throw new IllegalArgumentException(
+              "No column family : " + new String(column) + " available");
         }
         Collection<StoreFile> storeFiles = store.getStorefiles();
-        if (storeFiles == null) continue;
-        for (StoreFile storeFile: storeFiles) {
+        if (storeFiles == null) {
+          continue;
+        }
+        for (StoreFile storeFile : storeFiles) {
           storeFileNames.add(storeFile.getPath().toString());
         }
 
@@ -5368,7 +5328,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   //////////////////////////////////////////////////////////////////////////////
 
   /** Make sure this is a valid row for the HRegion */
-  void checkRow(final byte [] row, String op) throws IOException {
+  void checkRow(byte[] row, String op) throws IOException {
     if (!rowIsInRange(getRegionInfo(), row)) {
       throw new WrongRegionException("Requested row out of range for " +
           op + " on HRegion " + this + ", startKey='" +
@@ -5637,7 +5597,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         byte[] familyName = p.getFirst();
         String path = p.getSecond();
 
-        HStore store = getHStore(familyName);
+        HStore store = getStore(familyName);
         if (store == null) {
           IOException ioe = new org.apache.hadoop.hbase.DoNotRetryIOException(
               "No such column family " + Bytes.toStringBinary(familyName));
@@ -5697,7 +5657,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       for (Pair<byte[], String> p : familyPaths) {
         byte[] familyName = p.getFirst();
         String path = p.getSecond();
-        HStore store = getHStore(familyName);
+        HStore store = getStore(familyName);
         if (!familyWithFinalPath.containsKey(familyName)) {
           familyWithFinalPath.put(familyName, new ArrayList<>());
         }
@@ -5737,7 +5697,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         for (Pair<Path, Path> p : entry.getValue()) {
           String path = p.getFirst().toString();
           Path commitedStoreFile = p.getSecond();
-          HStore store = getHStore(familyName);
+          HStore store = getStore(familyName);
           try {
             store.bulkLoadHFile(familyName, path, commitedStoreFile);
             // Note the size of the store file
@@ -5912,7 +5872,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
       try {
         for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
-          Store store = stores.get(entry.getKey());
+          HStore store = stores.get(entry.getKey());
           KeyValueScanner scanner;
           try {
             scanner = store.getScanner(scan, entry.getValue(), this.readPt);
@@ -7145,7 +7105,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
                 // If no WAL, need to stamp it here.
                 CellUtil.setSequenceId(cell, sequenceId);
               }
-              applyToMemstore(getHStore(cell), cell, memstoreSize);
+              applyToMemstore(getStore(cell), cell, memstoreSize);
             }
           }
 
@@ -7296,7 +7256,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           return returnResults? cpResult: null;
         }
         Durability effectiveDurability = getEffectiveDurability(mutation.getDurability());
-        Map<Store, List<Cell>> forMemStore = new HashMap<>(mutation.getFamilyCellMap().size());
+        Map<HStore, List<Cell>> forMemStore = new HashMap<>(mutation.getFamilyCellMap().size());
         // Reckon Cells to apply to WAL --  in returned walEdit -- and what to add to memstore and
         // what to return back to the client (in 'forMemStore' and 'results' respectively).
         WALEdit walEdit = reckonDeltas(op, mutation, effectiveDurability, forMemStore, results);
@@ -7311,7 +7271,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           updateSequenceId(forMemStore.values(), writeEntry.getWriteNumber());
         }
         // Now write to MemStore. Do it a column family at a time.
-        for (Map.Entry<Store, List<Cell>> e : forMemStore.entrySet()) {
+        for (Map.Entry<HStore, List<Cell>> e : forMemStore.entrySet()) {
           applyToMemstore(e.getKey(), e.getValue(), true, memstoreSize);
         }
         mvcc.completeAndWait(writeEntry);
@@ -7419,18 +7379,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param forMemStore Fill in here what to apply to the MemStore (by Store).
    * @return A WALEdit to apply to WAL or null if we are to skip the WAL.
    */
-  private WALEdit reckonDeltas(final Operation op, final Mutation mutation,
-      final Durability effectiveDurability, final Map<Store, List<Cell>> forMemStore,
-      final List<Cell> results)
-  throws IOException {
+  private WALEdit reckonDeltas(Operation op, Mutation mutation, Durability effectiveDurability,
+      Map<HStore, List<Cell>> forMemStore, List<Cell> results) throws IOException {
     WALEdit walEdit = null;
     long now = EnvironmentEdgeManager.currentTime();
     final boolean writeToWAL = effectiveDurability != Durability.SKIP_WAL;
     // Process a Store/family at a time.
     for (Map.Entry<byte [], List<Cell>> entry: mutation.getFamilyCellMap().entrySet()) {
-      final byte [] columnFamilyName = entry.getKey();
+      final byte[] columnFamilyName = entry.getKey();
       List<Cell> deltas = entry.getValue();
-      Store store = this.stores.get(columnFamilyName);
+      HStore store = this.stores.get(columnFamilyName);
       // Reckon for the Store what to apply to WAL and MemStore.
       List<Cell> toApply =
         reckonDeltasByStore(store, op, mutation, effectiveDurability, now, deltas, results);
@@ -7462,11 +7420,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @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.
    */
-  private List<Cell> reckonDeltasByStore(final Store store, final Operation op,
-      final Mutation mutation, final Durability effectiveDurability, final long now,
-      final List<Cell> deltas, final List<Cell> results)
-  throws IOException {
-    byte [] columnFamily = store.getColumnFamilyDescriptor().getName();
+  private List<Cell> reckonDeltasByStore(HStore store, Operation op, Mutation mutation,
+      Durability effectiveDurability, long now, List<Cell> deltas, List<Cell> results)
+      throws IOException {
+    byte[] columnFamily = store.getColumnFamilyDescriptor().getName();
     List<Cell> toApply = new ArrayList<>(deltas.size());
     // Get previous values for all columns in this family.
     List<Cell> currentValues = get(mutation, store, deltas,
@@ -7576,9 +7533,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param coordinates Cells from <code>mutation</code> used as coordinates applied to Get.
    * @return Return list of Cells found.
    */
-  private List<Cell> get(final Mutation mutation, final Store store,
-          final List<Cell> coordinates, final IsolationLevel isolation, final TimeRange tr)
-  throws IOException {
+  private List<Cell> get(Mutation mutation, HStore store, List<Cell> coordinates,
+      IsolationLevel isolation, TimeRange tr) throws IOException {
     // Sort the cells so that they match the order that they appear in the Get results. Otherwise,
     // we won't be able to find the existing values if the cells are not specified in order by the
     // client since cells are in an array list.
@@ -7653,12 +7609,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @Override
   public long heapSize() {
-    long heapSize = DEEP_OVERHEAD;
-    for (Store store : this.stores.values()) {
-      heapSize += store.heapSize();
-    }
     // this does not take into account row locks, recent flushes, mvcc entries, and more
-    return heapSize;
+    return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum();
   }
 
   @Override
@@ -7813,14 +7765,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @return The priority that this region should have in the compaction queue
    */
   public int getCompactPriority() {
-    int count = Integer.MAX_VALUE;
-    for (Store store : stores.values()) {
-      count = Math.min(count, store.getCompactPriority());
-    }
-    return count;
+    return stores.values().stream().mapToInt(HStore::getCompactPriority).min()
+        .orElse(Store.NO_PRIORITY);
   }
 
-
   /** @return the coprocessor host */
   @Override
   public RegionCoprocessorHost getCoprocessorHost() {
@@ -7881,11 +7829,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // The unit for snapshot is a region. So, all stores for this region must be
     // prepared for snapshot operation before proceeding.
     if (op == Operation.SNAPSHOT) {
-      for (Store store : stores.values()) {
-        if (store instanceof HStore) {
-          ((HStore)store).preSnapshotOperation();
-        }
-      }
+      stores.values().forEach(HStore::preSnapshotOperation);
     }
     try {
       if (coprocessorHost != null) {
@@ -7905,11 +7849,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public void closeRegionOperation(Operation operation) throws IOException {
     if (operation == Operation.SNAPSHOT) {
-      for (Store store: stores.values()) {
-        if (store instanceof HStore) {
-          ((HStore)store).postSnapshotOperation();
-        }
-      }
+      stores.values().forEach(HStore::postSnapshotOperation);
     }
     lock.readLock().unlock();
     if (coprocessorHost != null) {
@@ -8142,9 +8082,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public void registerChildren(ConfigurationManager manager) {
     configurationManager = Optional.of(manager);
-    for (Store s : this.stores.values()) {
-      configurationManager.get().registerObserver(s);
-    }
+    stores.values().forEach(manager::registerObserver);
   }
 
   /**
@@ -8152,9 +8090,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   @Override
   public void deregisterChildren(ConfigurationManager manager) {
-    for (Store s : this.stores.values()) {
-      configurationManager.get().deregisterObserver(s);
-    }
+    stores.values().forEach(configurationManager.get()::deregisterObserver);
   }
 
   @Override
@@ -8175,7 +8111,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " ");
     buf.append(getRegionInfo().isMetaTable() ? " meta table " : " ");
     buf.append("stores: ");
-    for (Store s : getStores()) {
+    for (HStore s : stores.values()) {
       buf.append(s.getColumnFamilyDescriptor().getNameAsString());
       buf.append(" size: ");
       buf.append(s.getSizeOfMemStore().getDataSize());
@@ -8188,4 +8124,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       throw new RuntimeException(buf.toString());
     }
   }
+
+  @Override
+  public void requestCompaction(String why, int priority, CompactionLifeCycleTracker tracker,
+      User user) throws IOException {
+    ((HRegionServer) rsServices).compactSplitThread.requestCompaction(this, why, priority, tracker,
+      user);
+  }
+
+  @Override
+  public void requestCompaction(byte[] family, String why, int priority,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
+    ((HRegionServer) rsServices).compactSplitThread.requestCompaction(this,
+      Preconditions.checkNotNull(stores.get(family)), why, priority, tracker, user);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 6bbff36..62987c0 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
@@ -56,8 +56,8 @@ import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 import javax.servlet.http.HttpServlet;
 
-import org.apache.commons.lang3.SystemUtils;
 import org.apache.commons.lang3.RandomUtils;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -127,6 +127,7 @@ import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
@@ -140,6 +141,9 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@@ -210,10 +214,6 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-
 import sun.misc.Signal;
 import sun.misc.SignalHandler;
 
@@ -1686,7 +1686,7 @@ public class HRegionServer extends HasThread implements
     int totalStaticBloomSizeKB = 0;
     long totalCompactingKVs = 0;
     long currentCompactedKVs = 0;
-    List<Store> storeList = r.getStores();
+    List<? extends Store> storeList = r.getStores();
     stores += storeList.size();
     for (Store store : storeList) {
       storefiles += store.getStorefilesCount();
@@ -1772,27 +1772,32 @@ public class HRegionServer extends HasThread implements
     @Override
     protected void chore() {
       for (Region r : this.instance.onlineRegions.values()) {
-        if (r == null)
+        if (r == null) {
           continue;
-        for (Store s : r.getStores()) {
+        }
+        HRegion hr = (HRegion) r;
+        for (HStore s : hr.stores.values()) {
           try {
             long multiplier = s.getCompactionCheckMultiplier();
             assert multiplier > 0;
-            if (iteration % multiplier != 0) continue;
+            if (iteration % multiplier != 0) {
+              continue;
+            }
             if (s.needsCompaction()) {
               // Queue a compaction. Will recognize if major is needed.
-              this.instance.compactSplitThread.requestSystemCompaction(r, s, getName()
-                  + " requests compaction");
+              this.instance.compactSplitThread.requestSystemCompaction(hr, s,
+                getName() + " requests compaction");
             } else if (s.isMajorCompaction()) {
               s.triggerMajorCompaction();
-              if (majorCompactPriority == DEFAULT_PRIORITY
-                  || majorCompactPriority > ((HRegion)r).getCompactPriority()) {
-                this.instance.compactSplitThread.requestCompaction(r, s, getName()
-                    + " requests major compaction; use default priority", null);
+              if (majorCompactPriority == DEFAULT_PRIORITY ||
+                  majorCompactPriority > hr.getCompactPriority()) {
+                this.instance.compactSplitThread.requestCompaction(hr, s,
+                  getName() + " requests major compaction; use default priority", Store.NO_PRIORITY,
+                  CompactionLifeCycleTracker.DUMMY, null);
               } else {
-                this.instance.compactSplitThread.requestCompaction(r, s, getName()
-                    + " requests major compaction; use configured priority",
-                  this.majorCompactPriority, null, null);
+                this.instance.compactSplitThread.requestCompaction(hr, s,
+                  getName() + " requests major compaction; use configured priority",
+                  this.majorCompactPriority, CompactionLifeCycleTracker.DUMMY, null);
               }
             }
           } catch (IOException e) {
@@ -2146,15 +2151,14 @@ public class HRegionServer extends HasThread implements
   @Override
   public void postOpenDeployTasks(final PostOpenDeployContext context)
       throws KeeperException, IOException {
-    Region r = context.getRegion();
+    HRegion r = (HRegion) context.getRegion();
     long masterSystemTime = context.getMasterSystemTime();
-    Preconditions.checkArgument(r instanceof HRegion, "r must be an HRegion");
     rpcServices.checkOpen();
     LOG.info("Post open deploy tasks for " + r.getRegionInfo().getRegionNameAsString());
     // Do checks to see if we need to compact (references or too many files)
-    for (Store s : r.getStores()) {
+    for (HStore s : r.stores.values()) {
       if (s.hasReferences() || s.needsCompaction()) {
-       this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
+        this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
       }
     }
     long openSeqNum = r.getOpenSeqNum();
@@ -2863,11 +2867,6 @@ public class HRegionServer extends HasThread implements
     return serverName;
   }
 
-  @Override
-  public CompactionRequestor getCompactionRequester() {
-    return this.compactSplitThread;
-  }
-
   public RegionServerCoprocessorHost getRegionServerCoprocessorHost(){
     return this.rsHost;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 f011c18..daad241 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
@@ -30,6 +30,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
@@ -52,13 +53,12 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CompoundConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.FailedArchiveException;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
@@ -82,8 +83,6 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -92,14 +91,16 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 
 /**
  * A Store holds a column family in a Region.  Its a memstore and a set of zero
@@ -477,7 +478,7 @@ public class HStore implements Store {
   /**
    * @param tabledir {@link Path} to where the table is being stored
    * @param hri {@link HRegionInfo} for the region.
-   * @param family {@link HColumnDescriptor} describing the column family
+   * @param family {@link ColumnFamilyDescriptor} describing the column family
    * @return Path to family/Store home directory.
    */
   @Deprecated
@@ -489,7 +490,7 @@ public class HStore implements Store {
   /**
    * @param tabledir {@link Path} to where the table is being stored
    * @param encodedName Encoded region name.
-   * @param family {@link HColumnDescriptor} describing the column family
+   * @param family {@link ColumnFamilyDescriptor} describing the column family
    * @return Path to family/Store home directory.
    */
   @Deprecated
@@ -1386,15 +1387,14 @@ public class HStore implements Store {
     }
   }
 
-  private List<StoreFile> moveCompatedFilesIntoPlace(
-      final CompactionRequest cr, List<Path> newFiles, User user) throws IOException {
+  private List<StoreFile> moveCompatedFilesIntoPlace(CompactionRequest cr, List<Path> newFiles,
+      User user) throws IOException {
     List<StoreFile> sfs = new ArrayList<>(newFiles.size());
     for (Path newFile : newFiles) {
       assert newFile != null;
-      final StoreFile sf = moveFileIntoPlace(newFile);
+      StoreFile sf = moveFileIntoPlace(newFile);
       if (this.getCoprocessorHost() != null) {
-        final Store thisStore = this;
-        getCoprocessorHost().postCompact(thisStore, sf, cr, user);
+        getCoprocessorHost().postCompact(this, sf, cr.getTracker(), user);
       }
       assert sf != null;
       sfs.add(sf);
@@ -1636,23 +1636,12 @@ public class HStore implements Store {
   }
 
   @Override
-  public CompactionContext requestCompaction() throws IOException {
-    return requestCompaction(Store.NO_PRIORITY, null);
-  }
-
-  @Override
-  public CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
-      throws IOException {
-    return requestCompaction(priority, baseRequest, null);
-  }
-  @Override
-  public CompactionContext requestCompaction(int priority, final CompactionRequest baseRequest,
-      User user) throws IOException {
+  public Optional<CompactionContext> requestCompaction(int priority,
+      CompactionLifeCycleTracker tracker, User user) throws IOException {
     // don't even select for compaction if writes are disabled
     if (!this.areWritesEnabled()) {
-      return null;
+      return Optional.empty();
     }
-
     // Before we do compaction, try to get rid of unneeded files to simplify things.
     removeUnneededFiles();
 
@@ -1666,7 +1655,7 @@ public class HStore implements Store {
           final List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
           boolean override = false;
           override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc,
-              baseRequest, user);
+            tracker, user);
           if (override) {
             // Coprocessor is overriding normal file selection.
             compaction.forceSelect(new CompactionRequest(candidatesForCoproc));
@@ -1695,21 +1684,13 @@ public class HStore implements Store {
         }
         if (this.getCoprocessorHost() != null) {
           this.getCoprocessorHost().postCompactSelection(
-              this, ImmutableList.copyOf(compaction.getRequest().getFiles()), baseRequest, user);
-        }
-
-        // Selected files; see if we have a compaction with some custom base request.
-        if (baseRequest != null) {
-          // Update the request with what the system thinks the request should be;
-          // its up to the request if it wants to listen.
-          compaction.forceSelect(
-              baseRequest.combineWith(compaction.getRequest()));
+              this, ImmutableList.copyOf(compaction.getRequest().getFiles()), tracker, user);
         }
         // Finally, we have the resulting files list. Check if we have any files at all.
         request = compaction.getRequest();
-        final Collection<StoreFile> selectedFiles = request.getFiles();
+        Collection<StoreFile> selectedFiles = request.getFiles();
         if (selectedFiles.isEmpty()) {
-          return null;
+          return Optional.empty();
         }
 
         addToCompactingFiles(selectedFiles);
@@ -1721,6 +1702,7 @@ public class HStore implements Store {
         // Set priority, either override value supplied by caller or from store.
         request.setPriority((priority != Store.NO_PRIORITY) ? priority : getCompactPriority());
         request.setDescription(getRegionInfo().getRegionNameAsString(), getColumnFamilyName());
+        request.setTracker(tracker);
       }
     } finally {
       this.lock.readLock().unlock();
@@ -1730,7 +1712,7 @@ public class HStore implements Store {
         + ": Initiating " + (request.isMajor() ? "major" : "minor") + " compaction"
         + (request.isAllFiles() ? " (all files)" : ""));
     this.region.reportCompactionRequestStart(request.isMajor());
-    return compaction;
+    return Optional.of(compaction);
   }
 
   /** Adds the files to compacting files. filesCompacting must be locked. */

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 020142d..8fa686c 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
@@ -20,11 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.util.StringUtils.humanReadableInt;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
-import java.lang.management.MemoryType;
 import java.util.ArrayList;
 import java.util.ConcurrentModificationException;
 import java.util.HashMap;
@@ -50,6 +47,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.HasThread;
@@ -448,8 +446,8 @@ class MemStoreFlusher implements FlushRequester {
             "store files; delaying flush up to " + this.blockingWaitTime + "ms");
           if (!this.server.compactSplitThread.requestSplit(region)) {
             try {
-              this.server.compactSplitThread.requestSystemCompaction(
-                  region, Thread.currentThread().getName());
+              this.server.compactSplitThread.requestSystemCompaction((HRegion) region,
+                Thread.currentThread().getName());
             } catch (IOException e) {
               e = e instanceof RemoteException ?
                       ((RemoteException)e).unwrapRemoteException() : e;
@@ -503,8 +501,8 @@ class MemStoreFlusher implements FlushRequester {
       if (shouldSplit) {
         this.server.compactSplitThread.requestSplit(region);
       } else if (shouldCompact) {
-        server.compactSplitThread.requestSystemCompaction(
-            region, Thread.currentThread().getName());
+        server.compactSplitThread.requestSystemCompaction((HRegion) region,
+          Thread.currentThread().getName());
       }
     } catch (DroppedSnapshotException ex) {
       // Cache flush can fail in a few places. If it fails in a critical

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 2611f69..e30ed8e 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
@@ -761,7 +761,7 @@ class MetricsRegionServerWrapperImpl
           tempCheckAndMutateChecksFailed += r.getCheckAndMutateChecksFailed();
           tempCheckAndMutateChecksPassed += r.getCheckAndMutateChecksPassed();
           tempBlockedRequestsCount += r.getBlockedRequestsCount();
-          List<Store> storeList = r.getStores();
+          List<? extends Store> storeList = r.getStores();
           tempNumStores += storeList.size();
           for (Store store : storeList) {
             tempNumStoreFiles += store.getStorefilesCount();

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
index 667b46c..dc7d3cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
@@ -95,7 +95,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
 
   @Override
   public long getNumStores() {
-    Map<byte[],Store> stores = this.region.stores;
+    Map<byte[], HStore> stores = this.region.stores;
     if (stores == null) {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/61d10fef/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 02662c4..61c725b 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
@@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.regionserver.Leases.Lease;
 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
@@ -1538,7 +1539,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     try {
       checkOpen();
       requestCount.increment();
-      Region region = getRegion(request.getRegion());
+      HRegion region = (HRegion) getRegion(request.getRegion());
       // Quota support is enabled, the requesting user is not system/super user
       // and a quota policy is enforced that disables compactions.
       if (QuotaUtil.isQuotaEnabled(getConfiguration()) &&
@@ -1552,7 +1553,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString());
       boolean major = false;
       byte [] family = null;
-      Store store = null;
+      HStore store = null;
       if (request.hasFamily()) {
         family = request.getFamily().toByteArray();
         store = region.getStore(family);
@@ -1579,12 +1580,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           + region.getRegionInfo().getRegionNameAsString() + familyLogMsg);
       }
       String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
-      if(family != null) {
-        regionServer.compactSplitThread.requestCompaction(region, store, log,
-          Store.PRIORITY_USER, null, RpcServer.getRequestUser());
+      if (family != null) {
+        regionServer.compactSplitThread.requestCompaction(region, store, log, Store.PRIORITY_USER,
+          CompactionLifeCycleTracker.DUMMY, RpcServer.getRequestUser());
       } else {
-        regionServer.compactSplitThread.requestCompaction(region, log,
-          Store.PRIORITY_USER, null, RpcServer.getRequestUser());
+        regionServer.compactSplitThread.requestCompaction(region, log, Store.PRIORITY_USER,
+          CompactionLifeCycleTracker.DUMMY, RpcServer.getRequestUser());
       }
       return CompactRegionResponse.newBuilder().build();
     } catch (IOException ie) {
@@ -1606,7 +1607,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     try {
       checkOpen();
       requestCount.increment();
-      Region region = getRegion(request.getRegion());
+      HRegion region = (HRegion) getRegion(request.getRegion());
       LOG.info("Flushing " + region.getRegionInfo().getRegionNameAsString());
       boolean shouldFlush = true;
       if (request.hasIfOlderThanTs()) {
@@ -1617,8 +1618,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         boolean writeFlushWalMarker =  request.hasWriteFlushWalMarker() ?
             request.getWriteFlushWalMarker() : false;
         // Go behind the curtain so we can manage writing of the flush WAL marker
-        HRegion.FlushResultImpl flushResult = (HRegion.FlushResultImpl)
-            ((HRegion)region).flushcache(true, writeFlushWalMarker);
+        HRegion.FlushResultImpl flushResult = region.flushcache(true, writeFlushWalMarker);
         boolean compactionNeeded = flushResult.isCompactionNeeded();
         if (compactionNeeded) {
           regionServer.compactSplitThread.requestSystemCompaction(region,


[30/47] hbase git commit: HBASE-15931 Add log for long-running tasks in AsyncProcess (addendum)

Posted by bu...@apache.org.
HBASE-15931 Add log for long-running tasks in AsyncProcess (addendum)


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

Branch: refs/heads/HBASE-18467
Commit: b56760c732f4f72fcc3734cb0133262198d0889c
Parents: 34d6488
Author: Yu Li <li...@apache.org>
Authored: Fri Sep 8 11:38:00 2017 +0800
Committer: Yu Li <li...@apache.org>
Committed: Tue Sep 19 09:54:51 2017 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b56760c7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index f2d8d74..5e0da59 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -448,7 +448,7 @@ class AsyncProcess {
       LOG.info("#" + id + (max < 0 ? ", waiting for any free slot"
       : ", waiting for some tasks to finish. Expected max="
       + max) + ", tasksInProgress=" + currentInProgress +
-      " hasError=" + hasError() + tableName == null ? "" : ", tableName=" + tableName);
+      " hasError=" + hasError() + (tableName == null ? "" : ", tableName=" + tableName));
     };
   }
 


[02/47] hbase git commit: HBASE-18794 Remove deprecated methods in MasterObserver

Posted by bu...@apache.org.
HBASE-18794 Remove deprecated methods in MasterObserver


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

Branch: refs/heads/HBASE-18467
Commit: 38e983ed4484f60cbfc426c4ec7128c87a145096
Parents: d2792a0
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Wed Sep 13 10:34:44 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Sep 14 15:10:20 2017 +0800

----------------------------------------------------------------------
 .../hbase/coprocessor/MasterObserver.java       | 484 -------------------
 .../hbase/master/MasterCoprocessorHost.java     |  36 --
 .../hbase/security/access/AccessController.java |   6 +-
 .../hbase/coprocessor/TestMasterObserver.java   | 181 -------
 4 files changed, 3 insertions(+), 704 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/38e983ed/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 2f48682..9d649cc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -25,9 +25,7 @@ import java.util.Set;
 
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
@@ -105,44 +103,9 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called before a new table is created by
    * {@link org.apache.hadoop.hbase.master.HMaster}.  Called as part of create
-   * table handler and it is async to the create RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param desc the read-only HTableDescriptor for the table
-   * @param regions the initial regions created for the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preCreateTableAction(ObserverContext, TableDescriptor, HRegionInfo[])}.
-   */
-  @Deprecated
-  default void preCreateTableHandler(final ObserverContext<MasterCoprocessorEnvironment>
-      ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
-
-  /**
-   * Called after the createTable operation has been requested.  Called as part
-   * of create table RPC call.  Called as part of create table handler and
-   * it is async to the create RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param desc the read-only HTableDescriptor for the table
-   * @param regions the initial regions created for the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *   (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *   Use {@link #postCompletedCreateTableAction(ObserverContext, TableDescriptor, HRegionInfo[])}
-   */
-  @Deprecated
-  default void postCreateTableHandler(final ObserverContext<MasterCoprocessorEnvironment>
-  ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
-
-  /**
-   * Called before a new table is created by
-   * {@link org.apache.hadoop.hbase.master.HMaster}.  Called as part of create
    * table procedure and it is async to the create RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preCreateTableHandler(ObserverContext, HTableDescriptor, HRegionInfo[])} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param desc the TableDescriptor for the table
    * @param regions the initial regions created for the table
@@ -157,10 +120,6 @@ public interface MasterObserver extends Coprocessor {
    * of create table RPC call.  Called as part of create table procedure and
    * it is async to the create RPC call.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postCreateTableHandler(ObserverContext, HTableDescriptor, HRegionInfo[])} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param desc the TableDescriptor for the table
    * @param regions the initial regions created for the table
@@ -191,46 +150,10 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
-   * table.  Called as part of delete table handler and
-   * it is async to the delete RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preDeleteTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void preDeleteTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-      throws IOException {}
-
-  /**
-   * Called after {@link org.apache.hadoop.hbase.master.HMaster} deletes a
-   * table.  Called as part of delete table handler and it is async to the
-   * delete RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #postCompletedDeleteTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void postDeleteTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-      throws IOException {}
-
-  /**
-   * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
    * table.  Called as part of delete table procedure and
    * it is async to the delete RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preDeleteTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -244,10 +167,6 @@ public interface MasterObserver extends Coprocessor {
    * delete RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postDeleteTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -278,46 +197,10 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
-   * table.  Called as part of truncate table handler and it is sync
-   * to the truncate RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preTruncateTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void preTruncateTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-      throws IOException {}
-
-  /**
-   * Called after {@link org.apache.hadoop.hbase.master.HMaster} truncates a
-   * table.  Called as part of truncate table handler and it is sync to the
-   * truncate RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #postCompletedTruncateTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void postTruncateTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-      throws IOException {}
-
-  /**
-   * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
    * table.  Called as part of truncate table procedure and it is async
    * to the truncate RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preTruncateTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -331,10 +214,6 @@ public interface MasterObserver extends Coprocessor {
    * truncate RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postTruncateTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -365,45 +244,9 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called prior to modifying a table's properties.  Called as part of modify
-   * table handler and it is async to the modify table RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param htd the read-only HTableDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preModifyTableAction(ObserverContext, TableName, TableDescriptor)}.
-   */
-  @Deprecated
-  default void preModifyTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, HTableDescriptor htd) throws IOException {}
-
-  /**
-   * Called after to modifying a table's properties.  Called as part of modify
-   * table handler and it is async to the modify table RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param htd the read-only HTableDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *     Use {@link #postCompletedModifyTableAction(ObserverContext, TableName, TableDescriptor)}.
-   */
-  @Deprecated
-  default void postModifyTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, HTableDescriptor htd) throws IOException {}
-
-  /**
-   * Called prior to modifying a table's properties.  Called as part of modify
    * table procedure and it is async to the modify table RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preModifyTableHandler(ObserverContext, TableName, HTableDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param htd the TableDescriptor
@@ -418,10 +261,6 @@ public interface MasterObserver extends Coprocessor {
    * table procedure and it is async to the modify table RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postModifyTableHandler(ObserverContext, TableName, HTableDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param htd the TableDescriptor
@@ -434,24 +273,6 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called prior to adding a new column family to the table.  Called as part of
    * add column RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #preAddColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void preAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called prior to adding a new column family to the table.  Called as part of
-   * add column RPC call.
-   *
-   * Implementation note: This replaces the deprecated
-   * {@link #preAddColumn(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
@@ -463,24 +284,6 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called after the new column family has been created.  Called as part of
    * add column RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #postAddColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void postAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called after the new column family has been created.  Called as part of
-   * add column RPC call.
-   *
-   * Implementation note: This replaces the deprecated
-   * {@link #postAddColumn(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
@@ -491,27 +294,8 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called prior to adding a new column family to the table.  Called as part of
-   * add column handler.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *          (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
-   *          {@link #preAddColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void preAddColumnHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called prior to adding a new column family to the table.  Called as part of
    * add column procedure.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preAddColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param columnFamily the ColumnFamilyDescriptor
@@ -523,27 +307,8 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called after the new column family has been created.  Called as part of
-   * add column handler.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
-   *     {@link #postCompletedAddColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void postAddColumnHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called after the new column family has been created.  Called as part of
    * add column procedure.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postAddColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param columnFamily the ColumnFamilyDescriptor
@@ -556,24 +321,6 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called prior to modifying a column family's attributes.  Called as part of
    * modify column RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #preModifyColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void preModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called prior to modifying a column family's attributes.  Called as part of
-   * modify column RPC call.
-   *
-   * Implementation note: This replaces the deprecated
-   * {@link #preModifyColumn(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
@@ -585,24 +332,6 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called after the column family has been updated.  Called as part of modify
    * column RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #postModifyColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void postModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called after the column family has been updated.  Called as part of modify
-   * column RPC call.
-   *
-   * Implementation note: This replaces the deprecated
-   * {@link #postModifyColumn(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
@@ -613,27 +342,8 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called prior to modifying a column family's attributes.  Called as part of
-   * modify column handler.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *     Use {@link #preModifyColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void preModifyColumnHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called prior to modifying a column family's attributes.  Called as part of
    * modify column procedure.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preModifyColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param columnFamily the ColumnFamilyDescriptor
@@ -645,27 +355,8 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called after the column family has been updated.  Called as part of modify
-   * column handler.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the read-only HColumnDescriptor
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *   (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
-   *   {@link #postCompletedModifyColumnFamilyAction(ObserverContext,TableName,ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void postModifyColumnHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
-
-  /**
-   * Called after the column family has been updated.  Called as part of modify
    * column procedure.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postModifyColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param columnFamily the ColumnFamilyDescriptor
@@ -678,24 +369,6 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called prior to deleting the entire column family.  Called as part of
    * delete column RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the column family
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #preDeleteColumnFamily(ObserverContext, TableName, byte[])}.
-   */
-  @Deprecated
-  default void preDeleteColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, final byte[] columnFamily) throws IOException {}
-
-  /**
-   * Called prior to deleting the entire column family.  Called as part of
-   * delete column RPC call.
-   *
-   * Implementation note: This replaces the deprecated
-   * {@link #preDeleteColumn(ObserverContext, TableName, byte[])} method.
-   * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
@@ -707,24 +380,6 @@ public interface MasterObserver extends Coprocessor {
   /**
    * Called after the column family has been deleted.  Called as part of delete
    * column RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the column family
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #postDeleteColumnFamily(ObserverContext, TableName, byte[])}.
-   */
-  @Deprecated
-  default void postDeleteColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, final byte[] columnFamily) throws IOException {}
-
-  /**
-   * Called after the column family has been deleted.  Called as part of delete
-   * column RPC call.
-   *
-   * Implementation note: This replaces the deprecated
-   * {@link #postDeleteColumn(ObserverContext, TableName, byte[])} method.
-   * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
@@ -735,27 +390,8 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called prior to deleting the entire column family.  Called as part of
-   * delete column handler.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the column family
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #preDeleteColumnFamilyAction(ObserverContext, TableName, byte[])}.
-   */
-  @Deprecated
-  default void preDeleteColumnHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, final byte[] columnFamily) throws IOException {}
-
-  /**
-   * Called prior to deleting the entire column family.  Called as part of
    * delete column procedure.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preDeleteColumnHandler(ObserverContext, TableName, byte[])} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param columnFamily the column family
@@ -766,27 +402,8 @@ public interface MasterObserver extends Coprocessor {
 
   /**
    * Called after the column family has been deleted.  Called as part of
-   * delete column handler.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @param columnFamily the column family
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *         (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *         Use {@link #postCompletedDeleteColumnFamilyAction(ObserverContext, TableName, byte[])}.
-   */
-  @Deprecated
-  default void postDeleteColumnHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, final byte[] columnFamily) throws IOException {}
-
-  /**
-   * Called after the column family has been deleted.  Called as part of
    * delete column procedure.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postDeleteColumnHandler(ObserverContext, TableName, byte[])} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    * @param columnFamily the column family
@@ -814,43 +431,10 @@ public interface MasterObserver extends Coprocessor {
       final TableName tableName) throws IOException {}
 
   /**
-   * Called prior to enabling a table.  Called as part of enable table handler
-   * and it is async to the enable table RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preEnableTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void preEnableTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName) throws IOException {}
-
-  /**
-   * Called after the enableTable operation has been requested.  Called as part
-   * of enable table handler and it is async to the enable table RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #postCompletedEnableTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void postEnableTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName) throws IOException {}
-
-  /**
    * Called prior to enabling a table.  Called as part of enable table procedure
    * and it is async to the enable table RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preEnableTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -862,10 +446,6 @@ public interface MasterObserver extends Coprocessor {
    * Called after the enableTable operation has been requested.  Called as part
    * of enable table procedure and it is async to the enable table RPC call.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postEnableTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -893,43 +473,10 @@ public interface MasterObserver extends Coprocessor {
       final TableName tableName) throws IOException {}
 
   /**
-   * Called prior to disabling a table.  Called as part of disable table handler
-   * and it is asyn to the disable table RPC call.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preDisableTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void preDisableTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName) throws IOException {}
-
-  /**
-   * Called after the disableTable operation has been requested.  Called as part
-   * of disable table handler and it is asyn to the disable table RPC call.
-   * @param ctx the environment to interact with the framework and master
-   * @param tableName the name of the table
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #postCompletedDisableTableAction(ObserverContext, TableName)}.
-   */
-  @Deprecated
-  default void postDisableTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName) throws IOException {}
-
-  /**
    * Called prior to disabling a table.  Called as part of disable table procedure
    * and it is asyn to the disable table RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #preDisableTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -941,10 +488,6 @@ public interface MasterObserver extends Coprocessor {
    * Called after the disableTable operation has been requested.  Called as part
    * of disable table procedure and it is asyn to the disable table RPC call.
    *
-   * Implementation note: This replaces the deprecated
-   * {@link #postDisableTableHandler(ObserverContext, TableName)} method.
-   * Make sure to implement only one of the two as both are called.
-   *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
    */
@@ -1623,33 +1166,6 @@ public interface MasterObserver extends Coprocessor {
       final String namespace, final Quotas quotas) throws IOException {}
 
   /**
-   * Called before dispatching region merge request.
-   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
-   * @param ctx coprocessor environment
-   * @param regionA first region to be merged
-   * @param regionB second region to be merged
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *   (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
-   *   Use {@link #preMergeRegions(ObserverContext, HRegionInfo[])}
-   */
-  @Deprecated
-  default void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HRegionInfo regionA, HRegionInfo regionB) throws IOException {}
-
-  /**
-   * called after dispatching the region merge request.
-   * @param c coprocessor environment
-   * @param regionA first region to be merged
-   * @param regionB second region to be merged
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *   (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
-   *   Use {@link #postMergeRegions(ObserverContext, HRegionInfo[])}
-   */
-  @Deprecated
-  default void postDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> c,
-      final HRegionInfo regionA, final HRegionInfo regionB) throws IOException {}
-
-  /**
    * Called before merge regions request.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx coprocessor environment

http://git-wip-us.apache.org/repos/asf/hbase/blob/38e983ed/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 806742b..f0ce714 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -28,17 +28,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ImmutableHColumnDescriptor;
-import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -275,7 +271,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preCreateTableHandler(ctx, toImmutableHTableDescriptor(htd), regions);
         oserver.preCreateTableAction(ctx, htd, regions);
       }
     });
@@ -287,7 +282,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postCreateTableHandler(ctx, toImmutableHTableDescriptor(htd), regions);
         oserver.postCompletedCreateTableAction(ctx, htd, regions);
       }
     });
@@ -318,7 +312,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preDeleteTableHandler(ctx, tableName);
         oserver.preDeleteTableAction(ctx, tableName);
       }
     });
@@ -330,7 +323,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postDeleteTableHandler(ctx, tableName);
         oserver.postCompletedDeleteTableAction(ctx, tableName);
       }
     });
@@ -361,7 +353,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preTruncateTableHandler(ctx, tableName);
         oserver.preTruncateTableAction(ctx, tableName);
       }
     });
@@ -373,7 +364,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postTruncateTableHandler(ctx, tableName);
         oserver.postCompletedTruncateTableAction(ctx, tableName);
       }
     });
@@ -408,7 +398,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preModifyTableHandler(ctx, tableName, toImmutableHTableDescriptor(htd));
         oserver.preModifyTableAction(ctx, tableName, htd);
       }
     });
@@ -421,7 +410,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postModifyTableHandler(ctx, tableName, toImmutableHTableDescriptor(htd));
         oserver.postCompletedModifyTableAction(ctx, tableName, htd);
       }
     });
@@ -433,7 +421,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preAddColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preAddColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -445,7 +432,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postAddColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postAddColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -460,7 +446,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preAddColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preAddColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -475,7 +460,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postAddColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postCompletedAddColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -487,7 +471,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preModifyColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preModifyColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -499,7 +482,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postModifyColumn(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postModifyColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -513,7 +495,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preModifyColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.preModifyColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -527,7 +508,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postModifyColumnHandler(ctx, tableName, toImmutableHColumnDescriptor(columnFamily));
         oserver.postCompletedModifyColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -539,7 +519,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preDeleteColumn(ctx, tableName, columnFamily);
         oserver.preDeleteColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -551,7 +530,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postDeleteColumn(ctx, tableName, columnFamily);
         oserver.postDeleteColumnFamily(ctx, tableName, columnFamily);
       }
     });
@@ -566,7 +544,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preDeleteColumnHandler(ctx, tableName, columnFamily);
         oserver.preDeleteColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -578,7 +555,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postDeleteColumnHandler(ctx, tableName, columnFamily);
         oserver.postCompletedDeleteColumnFamilyAction(ctx, tableName, columnFamily);
       }
     });
@@ -609,7 +585,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preEnableTableHandler(ctx, tableName);
         oserver.preEnableTableAction(ctx, tableName);
       }
     });
@@ -621,7 +596,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postEnableTableHandler(ctx, tableName);
         oserver.postCompletedEnableTableAction(ctx, tableName);
       }
     });
@@ -652,7 +626,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.preDisableTableHandler(ctx, tableName);
         oserver.preDisableTableAction(ctx, tableName);
       }
     });
@@ -664,7 +637,6 @@ public class MasterCoprocessorHost
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
           throws IOException {
-        oserver.postDisableTableHandler(ctx, tableName);
         oserver.postCompletedDisableTableAction(ctx, tableName);
       }
     });
@@ -1918,12 +1890,4 @@ public class MasterCoprocessorHost
       }
     });
   }
-
-  private static ImmutableHTableDescriptor toImmutableHTableDescriptor(TableDescriptor desc) {
-    return new ImmutableHTableDescriptor(desc);
-  }
-
-  private static ImmutableHColumnDescriptor toImmutableHColumnDescriptor(ColumnFamilyDescriptor desc) {
-    return new ImmutableHColumnDescriptor(desc);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/38e983ed/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 931386e..f7d70d2 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
@@ -2566,9 +2566,9 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   }
 
   @Override
-  public void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HRegionInfo regionA, HRegionInfo regionB) throws IOException {
-    requirePermission(getActiveUser(ctx), "mergeRegions", regionA.getTable(), null, null,
+  public void preMergeRegions(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              final HRegionInfo[] regionsToMerge) throws IOException {
+    requirePermission(getActiveUser(ctx), "mergeRegions", regionsToMerge[0].getTable(), null, null,
       Action.ADMIN);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/38e983ed/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 49ab418..fbfeb50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -181,8 +181,6 @@ public class TestMasterObserver {
     private boolean postGetTableDescriptorsCalled;
     private boolean postGetTableNamesCalled;
     private boolean preGetTableNamesCalled;
-    private boolean preDispatchMergeCalled;
-    private boolean postDispatchMergeCalled;
     private boolean preMergeRegionsCalled;
     private boolean postMergeRegionsCalled;
     private boolean preRequestLockCalled;
@@ -275,8 +273,6 @@ public class TestMasterObserver {
       postGetTableDescriptorsCalled = false;
       postGetTableNamesCalled = false;
       preGetTableNamesCalled = false;
-      preDispatchMergeCalled = false;
-      postDispatchMergeCalled = false;
       preMergeRegionsCalled = false;
       postMergeRegionsCalled = false;
       preRequestLockCalled = false;
@@ -304,22 +300,6 @@ public class TestMasterObserver {
     }
 
     @Override
-    public void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        HRegionInfo regionA, HRegionInfo regionB) throws IOException {
-      preDispatchMergeCalled = true;
-    }
-
-    @Override
-    public void postDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        HRegionInfo regionA, HRegionInfo regionB) throws IOException {
-      postDispatchMergeCalled = true;
-    }
-
-    public boolean wasDispatchMergeCalled() {
-      return preDispatchMergeCalled && postDispatchMergeCalled;
-    }
-
-    @Override
     public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
         TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       if (bypass) {
@@ -533,13 +513,6 @@ public class TestMasterObserver {
       return preListNamespaceDescriptorsCalled && !postListNamespaceDescriptorsCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily
-    ) throws IOException {
-    }
-
     @Override
     public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
         TableName tableName, ColumnFamilyDescriptor columnFamily
@@ -553,12 +526,6 @@ public class TestMasterObserver {
       preAddColumnCalled = true;
     }
 
-    @Deprecated
-    @Override
-    public void postAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily) throws IOException {
-    }
-
     @Override
     public void postAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
         TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
@@ -573,12 +540,6 @@ public class TestMasterObserver {
       return preAddColumnCalled && !postAddColumnCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily) throws IOException {
-    }
-
     @Override
     public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
         TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
@@ -588,12 +549,6 @@ public class TestMasterObserver {
       preModifyColumnCalled = true;
     }
 
-    @Deprecated
-    @Override
-    public void postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily) throws IOException {
-    }
-
     @Override
     public void postModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
         TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
@@ -608,12 +563,6 @@ public class TestMasterObserver {
       return preModifyColumnCalled && !postModifyColumnCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, byte[] columnFamily) throws IOException {
-    }
-
     @Override
     public void preDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
         TableName tableName, byte[] columnFamily) throws IOException {
@@ -623,12 +572,6 @@ public class TestMasterObserver {
       preDeleteColumnCalled = true;
     }
 
-    @Deprecated
-    @Override
-    public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, byte[] columnFamily) throws IOException {
-    }
-
     @Override
     public void postDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
         TableName tableName, byte[] columnFamily) throws IOException {
@@ -1030,13 +973,6 @@ public class TestMasterObserver {
       return preDeleteSnapshotCalled && postDeleteSnapshotCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preCreateTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> env,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
-    }
-
     @Override
     public void preCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env,
@@ -1048,13 +984,6 @@ public class TestMasterObserver {
       preCreateTableActionCalled = true;
     }
 
-    @Deprecated
-    @Override
-    public void postCreateTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
-    }
-
     @Override
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -1075,20 +1004,6 @@ public class TestMasterObserver {
       return preCreateTableActionCalled && !postCompletedCreateTableActionCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preDeleteTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName)
-        throws IOException {
-    }
-
-    @Deprecated
-    @Override
-    public void postDeleteTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-        throws IOException {
-    }
-
     @Override
     public void preDeleteTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env, final TableName tableName)
@@ -1115,20 +1030,6 @@ public class TestMasterObserver {
       return preDeleteTableActionCalled && !postCompletedDeleteTableActionCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preTruncateTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName)
-        throws IOException {
-    }
-
-    @Deprecated
-    @Override
-    public void postTruncateTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-        throws IOException {
-    }
-
     @Override
     public void preTruncateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env, final TableName tableName)
@@ -1154,20 +1055,6 @@ public class TestMasterObserver {
       return preTruncateTableActionCalled && !postCompletedTruncateTableActionCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preModifyTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName,
-        HTableDescriptor htd) throws IOException {
-    }
-
-    @Deprecated
-    @Override
-    public void postModifyTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName,
-        HTableDescriptor htd) throws IOException {
-    }
-
     @Override
     public void preModifyTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env,
@@ -1195,13 +1082,6 @@ public class TestMasterObserver {
       return preModifyTableActionCalled && !postCompletedModifyTableActionCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preAddColumnHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
-        HColumnDescriptor columnFamily) throws IOException {
-    }
-
     @Override
     public void preAddColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -1213,13 +1093,6 @@ public class TestMasterObserver {
       preAddColumnFamilyActionCalled = true;
     }
 
-    @Deprecated
-    @Override
-    public void postAddColumnHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
-        HColumnDescriptor columnFamily) throws IOException {
-    }
-
     @Override
     public void postCompletedAddColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -1236,13 +1109,6 @@ public class TestMasterObserver {
       return preAddColumnFamilyActionCalled && !postCompletedAddColumnFamilyActionCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preModifyColumnHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
-        HColumnDescriptor columnFamily) throws IOException {
-    }
-
     @Override
     public void preModifyColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -1254,13 +1120,6 @@ public class TestMasterObserver {
       preModifyColumnFamilyActionCalled = true;
     }
 
-    @Deprecated
-    @Override
-    public void postModifyColumnHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
-        HColumnDescriptor columnFamily) throws IOException {
-    }
-
     @Override
     public void postCompletedModifyColumnFamilyAction(
         ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
@@ -1276,13 +1135,6 @@ public class TestMasterObserver {
       return preModifyColumnFamilyActionCalled && !postCompletedModifyColumnFamilyActionCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preDeleteColumnHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
-        byte[] columnFamily) throws IOException {
-    }
-
     @Override
     public void preDeleteColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -1294,13 +1146,6 @@ public class TestMasterObserver {
       preDeleteColumnFamilyActionCalled = true;
     }
 
-    @Deprecated
-    @Override
-    public void postDeleteColumnHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
-        byte[] columnFamily) throws IOException {
-    }
-
     @Override
     public void postCompletedDeleteColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -1317,20 +1162,6 @@ public class TestMasterObserver {
       return preDeleteColumnFamilyActionCalled && !postCompletedDeleteColumnFamilyActionCalled;
     }
 
-    @Deprecated
-    @Override
-    public void preEnableTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName)
-        throws IOException {
-    }
-
-    @Deprecated
-    @Override
-    public void postEnableTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-        throws IOException {
-    }
-
     @Override
     public void preEnableTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName)
@@ -1357,18 +1188,6 @@ public class TestMasterObserver {
     }
 
     @Override
-    public void preDisableTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName)
-        throws IOException {
-    }
-
-    @Override
-    public void postDisableTableHandler(
-        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
-        throws IOException {
-    }
-
-    @Override
     public void preDisableTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName)
         throws IOException {


[31/47] hbase git commit: HBASE-18851 Add "BSD" to the acceptable licenses list

Posted by bu...@apache.org.
HBASE-18851 Add "BSD" to the acceptable licenses list

org.ow2.asm:asm actually is 3-clause BSD, but it was
marked as "BSD" which we need to add to our exclusions
list.


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

Branch: refs/heads/HBASE-18467
Commit: a29ea36194a18c1de415577e78ca3553818a72b5
Parents: b56760c
Author: Josh Elser <el...@apache.org>
Authored: Tue Sep 19 15:18:39 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Sep 19 16:19:39 2017 -0400

----------------------------------------------------------------------
 hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a29ea361/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
index e1475a5..0df8f33 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
@@ -1338,7 +1338,7 @@ You can redistribute it and/or modify it under either the terms of the
 ## license mentions.
 ##
 ## See this FAQ link for justifications: https://www.apache.org/legal/resolved.html
-#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD license', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 2.0', 'Creative Commons Attribution License, Version 2.5' ])
+#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD license', 'BSD', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 2.0', 'Creative Commons Attribution License, Version 2.5' ])
 ## include LICENSE sections for anything not under ASL2.0
 #foreach( ${dep} in ${projects} )
 ## if there are no licenses we'll fail the build later, so


[06/47] hbase git commit: HBASE-18801 Bulk load cleanup may falsely deem file deletion successful

Posted by bu...@apache.org.
HBASE-18801 Bulk load cleanup may falsely deem file deletion successful

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


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

Branch: refs/heads/HBASE-18467
Commit: c17b49143259bb5a813a551b4eb9fa3e246a9f6e
Parents: 61d10fe
Author: Reid Chan <re...@outlook.com>
Authored: Thu Sep 14 20:28:17 2017 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Sep 14 08:09:42 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/SecureBulkLoadManager.java     | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c17b4914/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 85ccc91..66773d0 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
@@ -170,7 +170,13 @@ public class SecureBulkLoadManager {
       }
     }
 
-    fs.delete(new Path(request.getBulkToken()), true);
+    Path path = new Path(request.getBulkToken());
+    if (!fs.delete(path, true)) {
+      if (fs.exists(path)) {
+        throw new IOException("Failed to clean up " + path);
+      }
+    }
+    LOG.info("Cleaned up " + path + " successfully.");
   }
 
   public Map<byte[], List<Path>> secureBulkLoadHFiles(final Region region,


[19/47] hbase git commit: HBASE-18831 Add explicit dependency on javax.el

Posted by bu...@apache.org.
HBASE-18831 Add explicit dependency on javax.el

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


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

Branch: refs/heads/HBASE-18467
Commit: 5c12e424e679e738a5b6282ff24348a50ed4ca14
Parents: 81d2927
Author: Michael Stack <st...@apache.org>
Authored: Fri Sep 15 17:30:12 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Sat Sep 16 10:37:27 2017 -0700

----------------------------------------------------------------------
 hbase-rest/pom.xml   | 4 ++++
 hbase-server/pom.xml | 4 ++++
 hbase-thrift/pom.xml | 4 ++++
 pom.xml              | 6 ++++++
 4 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5c12e424/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index a0ac6f7..4553293 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -309,6 +309,10 @@
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.glassfish</groupId>
+      <artifactId>javax.el</artifactId>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hbase/blob/5c12e424/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 1d888d3..03891a1 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -496,6 +496,10 @@
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.glassfish</groupId>
+      <artifactId>javax.el</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.codehaus.jackson</groupId>
       <artifactId>jackson-core-asl</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/5c12e424/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index dc9cd8c..36c992d 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -299,6 +299,10 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.glassfish</groupId>
+      <artifactId>javax.el</artifactId>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/5c12e424/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index af98b51..a684f3c 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1408,6 +1408,7 @@
     <wx.rs.api.version>2.0.1</wx.rs.api.version>
     <jersey.version>2.25.1</jersey.version>
     <glassfish.jsp.version>2.3.2</glassfish.jsp.version>
+    <glassfish.el.version>3.0.1-b08</glassfish.el.version>
     <jetty.jspapi.version>6.1.14</jetty.jspapi.version>
     <jruby.version>9.1.10.0</jruby.version>
     <junit.version>4.12</junit.version>
@@ -1988,6 +1989,11 @@
         <version>${glassfish.jsp.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.glassfish</groupId>
+        <artifactId>javax.el</artifactId>
+        <version>${glassfish.el.version}</version>
+      </dependency>
+      <dependency>
         <groupId>javax.xml.bind</groupId>
         <artifactId>jaxb-api</artifactId>
         <version>${jaxb-api.version}</version>


[26/47] hbase git commit: HBASE-18142 Deletion of a cell deletes the previous versions too

Posted by bu...@apache.org.
HBASE-18142 Deletion of a cell deletes the previous versions too

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


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

Branch: refs/heads/HBASE-18467
Commit: ff0c1d2fa3addf92ed36139bfc6e21024049cb4f
Parents: 5264f04
Author: Chun-Hao Tang <ta...@gmail.com>
Authored: Fri Sep 15 02:13:12 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Tue Sep 19 00:22:59 2017 +0800

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/table.rb        | 30 ++++++++-----
 .../src/main/ruby/shell/commands/delete.rb      |  6 +--
 .../src/main/ruby/shell/commands/deleteall.rb   |  2 +-
 hbase-shell/src/test/ruby/hbase/table_test.rb   | 47 ++++++++------------
 4 files changed, 40 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ff0c1d2f/hbase-shell/src/main/ruby/hbase/table.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb
index b297f58..7eaf1b3 100644
--- a/hbase-shell/src/main/ruby/hbase/table.rb
+++ b/hbase-shell/src/main/ruby/hbase/table.rb
@@ -24,7 +24,6 @@ include Java
 module Hbase
   class Table
     include HBaseConstants
-
     @@thread_pool = nil
 
     # Add the command 'name' to table s.t. the shell command also called via 'name'
@@ -102,7 +101,7 @@ flush and drop just by typing:
 Note that after dropping a table, your reference to it becomes useless and further usage
 is undefined (and not recommended).
 EOF
-      end
+    end
 
     #---------------------------------------------------------------------------------------------
 
@@ -162,7 +161,8 @@ EOF
     #----------------------------------------------------------------------------------------------
     # Create a Delete mutation
     def _createdelete_internal(row, column = nil,
-                               timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP, args = {})
+                               timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP,
+                               args = {}, all_version = true)
       temptimestamp = timestamp
       if temptimestamp.is_a?(Hash)
         timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP
@@ -179,9 +179,12 @@ EOF
         visibility = args[VISIBILITY]
         set_cell_visibility(d, visibility) if visibility
       end
-      if column
+      if column && all_version
         family, qualifier = parse_column_name(column)
         d.addColumns(family, qualifier, timestamp)
+      elsif column && !all_version
+        family, qualifier = parse_column_name(column)
+        d.addColumn(family, qualifier, timestamp)
       end
       d
     end
@@ -189,7 +192,8 @@ EOF
     #----------------------------------------------------------------------------------------------
     # Delete rows using prefix
     def _deleterows_internal(row, column = nil,
-                             timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP, args = {})
+                             timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP,
+                             args = {}, all_version = true)
       cache = row['CACHE'] ? row['CACHE'] : 100
       prefix = row['ROWPREFIXFILTER']
 
@@ -205,7 +209,7 @@ EOF
       while iter.hasNext
         row = iter.next
         key = org.apache.hadoop.hbase.util.Bytes.toStringBinary(row.getRow)
-        d = _createdelete_internal(key, column, timestamp, args)
+        d = _createdelete_internal(key, column, timestamp, args, all_version)
         list.add(d)
         if list.size >= cache
           @table.delete(list)
@@ -218,23 +222,25 @@ EOF
     #----------------------------------------------------------------------------------------------
     # Delete a cell
     def _delete_internal(row, column,
-                         timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP, args = {})
-      _deleteall_internal(row, column, timestamp, args)
+                         timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP,
+                         args = {}, all_version = false)
+      _deleteall_internal(row, column, timestamp, args, all_version)
     end
 
     #----------------------------------------------------------------------------------------------
     # Delete a row
     def _deleteall_internal(row, column = nil,
-                            timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP, args = {})
+                            timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP,
+                            args = {}, all_version = true)
       # delete operation doesn't need read permission. Retaining the read check for
       # meta table as a part of HBASE-5837.
       if is_meta_table?
         raise ArgumentError, 'Row Not Found' if _get_internal(row).nil?
       end
       if row.is_a?(Hash)
-        _deleterows_internal(row, column, timestamp, args)
+        _deleterows_internal(row, column, timestamp, args, all_version)
       else
-        d = _createdelete_internal(row, column, timestamp, args)
+        d = _createdelete_internal(row, column, timestamp, args, all_version)
         @table.delete(d)
       end
     end
@@ -510,7 +516,7 @@ EOF
                  org.apache.hadoop.hbase.client.Scan.new(startrow.to_java_bytes, stoprow.to_java_bytes)
                else
                  org.apache.hadoop.hbase.client.Scan.new(startrow.to_java_bytes)
-        end
+               end
 
         # This will overwrite any startrow/stoprow settings
         scan.setRowPrefixFilter(rowprefixfilter.to_java_bytes) if rowprefixfilter

http://git-wip-us.apache.org/repos/asf/hbase/blob/ff0c1d2f/hbase-shell/src/main/ruby/shell/commands/delete.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/delete.rb b/hbase-shell/src/main/ruby/shell/commands/delete.rb
index 3e4447c..6995959 100644
--- a/hbase-shell/src/main/ruby/shell/commands/delete.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/delete.rb
@@ -40,15 +40,15 @@ t to table 't1', the corresponding command would be:
 EOF
       end
 
-      def command(table, row, column,
+      def command(table, row, column = nil,
                   timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP, args = {})
         delete(table(table), row, column, timestamp, args)
       end
 
-      def delete(table, row, column,
+      def delete(table, row, column = nil,
                  timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP, args = {})
         @start_time = Time.now
-        table._delete_internal(row, column, timestamp, args)
+        table._delete_internal(row, column, timestamp, args, false)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/ff0c1d2f/hbase-shell/src/main/ruby/shell/commands/deleteall.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/deleteall.rb b/hbase-shell/src/main/ruby/shell/commands/deleteall.rb
index f5444ae..f18fa05 100644
--- a/hbase-shell/src/main/ruby/shell/commands/deleteall.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/deleteall.rb
@@ -58,7 +58,7 @@ EOF
       def deleteall(table, row, column = nil,
                     timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP, args = {})
         @start_time = Time.now
-        table._deleteall_internal(row, column, timestamp, args)
+        table._deleteall_internal(row, column, timestamp, args, true)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/ff0c1d2f/hbase-shell/src/test/ruby/hbase/table_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/table_test.rb b/hbase-shell/src/test/ruby/hbase/table_test.rb
index a01514c..81d0a9a 100644
--- a/hbase-shell/src/test/ruby/hbase/table_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/table_test.rb
@@ -106,18 +106,12 @@ module Hbase
       @test_table = table(@test_name)
       
       # Insert data to perform delete operations
-      @test_table.put("101", "x:a", "1")
-      @test_table.put("101", "x:a", "2", Time.now.to_i)
-      
-      @test_table.put("102", "x:a", "1", 1212)
-      @test_table.put("102", "x:a", "2", 1213)
-      
-      @test_table.put(103, "x:a", "3")
-      @test_table.put(103, "x:a", "4")
-      
+      @test_table.put("102", "x:a", "2", 1212)
+      @test_table.put(103, "x:a", "3", 1214)
+
       @test_table.put("104", "x:a", 5)
       @test_table.put("104", "x:b", 6)
-      
+
       @test_table.put(105, "x:a", "3")
       @test_table.put(105, "x:a", "4")
 
@@ -152,21 +146,16 @@ module Hbase
     end
 
     #-------------------------------------------------------------------------------
-
-    define_test "delete should work without timestamp" do
-      @test_table.delete("101", "x:a")
-      res = @test_table._get_internal('101', 'x:a')
-      assert_nil(res)
-    end
-
-    define_test "delete should work with timestamp" do
-      @test_table.delete("102", "x:a", 1214)
+    define_test "delete should work with string keys" do
+      @test_table.delete('102', 'x:a', 1212)
       res = @test_table._get_internal('102', 'x:a')
       assert_nil(res)
     end
 
     define_test "delete should work with integer keys" do
-      @test_table.delete(103, "x:a")
+      res = @test_table._get_internal('103', 'x:a')
+      assert_not_nil(res)
+      @test_table.delete(103, 'x:a', 1214)
       res = @test_table._get_internal('103', 'x:a')
       assert_nil(res)
     end
@@ -266,7 +255,7 @@ module Hbase
         count = @test_table.count COLUMNS => [ 'x:c']
         assert(count == 1)
       ensure
-        @test_table.delete(4, "x:c")
+        @test_table.deleteall(4, 'x:c')
       end
     end
 
@@ -413,8 +402,8 @@ module Hbase
           assert_not_nil(/value=98/.match(res['x:d']))
         ensure
           # clean up newly added columns for this test only.
-          @test_table.delete(1, "x:c")
-          @test_table.delete(1, "x:d")
+          @test_table.deleteall(1, 'x:c')
+          @test_table.deleteall(1, 'x:d')
         end
     end
 
@@ -430,7 +419,7 @@ module Hbase
         assert_nil(res)
       ensure
         # clean up newly added columns for this test only.
-        @test_table.delete(1, "x:v")
+        @test_table.deleteall(1, 'x:v')
       end
     end
 
@@ -613,8 +602,8 @@ module Hbase
         assert_not_nil(/value=98/.match(res['1']['x:d']))
       ensure
         # clean up newly added columns for this test only.
-        @test_table.delete(1, "x:c")
-        @test_table.delete(1, "x:d")
+        @test_table.deleteall(1, 'x:c')
+        @test_table.deleteall(1, 'x:d')
       end
     end
 
@@ -632,7 +621,7 @@ module Hbase
         assert_equal(res, {}, "Result is not empty")
       ensure
         # clean up newly added columns for this test only.
-        @test_table.delete(1, "x:v")
+        @test_table.deleteall(1, 'x:v')
       end
     end
 
@@ -648,7 +637,7 @@ module Hbase
         assert_nil(res['2'])
       ensure
         # clean up newly added columns for this test only.
-        @test_table.delete(4, "x:a")
+        @test_table.deleteall(4, 'x:a')
       end
     end
 
@@ -666,7 +655,7 @@ module Hbase
         res = @test_table._get_internal('ttlTest', 'x:a')
         assert_nil(res)
       ensure
-        @test_table.delete('ttlTest', 'x:a')
+        @test_table.deleteall('ttlTest', 'x:a')
       end
     end
 


[24/47] hbase git commit: HBASE-18835 The return type of ExtendedCell#deepClone should be ExtendedCell rather than Cell

Posted by bu...@apache.org.
HBASE-18835 The return type of ExtendedCell#deepClone should be ExtendedCell rather than Cell


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

Branch: refs/heads/HBASE-18467
Commit: fc13cf7d7843029d807878869e3c64cace5640dd
Parents: efb95a1
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Mon Sep 18 18:22:55 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Sep 18 22:31:32 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/ByteBufferKeyValue.java     | 2 +-
 .../src/main/java/org/apache/hadoop/hbase/CellUtil.java      | 8 ++++----
 .../src/main/java/org/apache/hadoop/hbase/ExtendedCell.java  | 2 +-
 .../org/apache/hadoop/hbase/IndividualBytesFieldCell.java    | 2 +-
 .../src/main/java/org/apache/hadoop/hbase/KeyValue.java      | 2 +-
 .../org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java    | 2 +-
 .../main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java    | 2 +-
 .../hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java   | 4 ++--
 8 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
index f906681..c59b947 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
@@ -302,7 +302,7 @@ public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
   }
 
   @Override
-  public Cell deepClone() {
+  public ExtendedCell deepClone() {
     byte[] copy = new byte[this.length];
     ByteBufferUtils.copyFromBufferToArray(copy, this.buf, this.offset, 0, this.length);
     KeyValue kv = new KeyValue(copy, 0, copy.length);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 825d9b1..a3029f8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -657,7 +657,7 @@ public final class CellUtil {
     }
 
     @Override
-    public Cell deepClone() {
+    public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
       return new TagRewriteCell(clonedBaseCell, this.tags);
     }
@@ -838,7 +838,7 @@ public final class CellUtil {
     }
 
     @Override
-    public Cell deepClone() {
+    public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
       if (clonedBaseCell instanceof ByteBufferCell) {
         return new TagRewriteByteBufferCell((ByteBufferCell) clonedBaseCell, this.tags);
@@ -981,7 +981,7 @@ public final class CellUtil {
     }
 
     @Override
-    public Cell deepClone() {
+    public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
       return new ValueAndTagRewriteCell(clonedBaseCell, this.value, this.tags);
     }
@@ -1047,7 +1047,7 @@ public final class CellUtil {
     }
 
     @Override
-    public Cell deepClone() {
+    public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
       if (clonedBaseCell instanceof ByteBufferCell) {
         return new ValueAndTagRewriteByteBufferCell((ByteBufferCell) clonedBaseCell, this.value,

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
index 7ed4dc0..4d16fca 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
@@ -73,7 +73,7 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
    * Does a deep copy of the contents to a new memory area and returns it as a new cell.
    * @return The deep cloned cell
    */
-  Cell deepClone();
+  ExtendedCell deepClone();
 
   /**
    * Extracts the id of the backing bytebuffer of this cell if it was obtained from fixed sized

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
index 0597c5e..14e35df 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
@@ -183,7 +183,7 @@ public class IndividualBytesFieldCell implements ExtendedCell {
   }
 
   @Override
-  public Cell deepClone() {
+  public ExtendedCell deepClone() {
     // When being added to the memstore, deepClone() is called and KeyValue has less heap overhead.
     return new KeyValue(this);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/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 0ee8b80..ae95738 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
@@ -2808,7 +2808,7 @@ public class KeyValue implements ExtendedCell {
   }
 
   @Override
-  public Cell deepClone() {
+  public ExtendedCell deepClone() {
     byte[] copy = Bytes.copy(this.bytes, this.offset, this.length);
     KeyValue kv = new KeyValue(copy, 0, copy.length);
     kv.setSequenceId(this.getSequenceId());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java
index 1822563..82b243b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java
@@ -52,7 +52,7 @@ public class NoTagsByteBufferKeyValue extends ByteBufferKeyValue {
   }
 
   @Override
-  public Cell deepClone() {
+  public ExtendedCell deepClone() {
     byte[] copy = new byte[this.length];
     ByteBufferUtils.copyFromBufferToArray(copy, this.buf, this.offset, 0, this.length);
     KeyValue kv = new NoTagsKeyValue(copy, 0, copy.length);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
index 8a57a01..088aff5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsKeyValue.java
@@ -51,7 +51,7 @@ public class NoTagsKeyValue extends KeyValue {
   }
 
   @Override
-  public Cell deepClone() {
+  public ExtendedCell deepClone() {
     byte[] copy = Bytes.copy(this.bytes, this.offset, this.length);
     KeyValue kv = new NoTagsKeyValue(copy, 0, copy.length);
     kv.setSequenceId(this.getSequenceId());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc13cf7d/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index 00ec0fc..bc905e5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -470,7 +470,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
     }
 
     @Override
-    public Cell deepClone() {
+    public ExtendedCell deepClone() {
       // This is not used in actual flow. Throwing UnsupportedOperationException
       throw new UnsupportedOperationException();
     }
@@ -715,7 +715,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
     }
 
     @Override
-    public Cell deepClone() {
+    public ExtendedCell deepClone() {
       // This is not used in actual flow. Throwing UnsupportedOperationException
       throw new UnsupportedOperationException();
     }


[18/47] hbase git commit: HBASE-17980 (Addendum) make UnmodifyableHRegionInfo deprecated

Posted by bu...@apache.org.
HBASE-17980 (Addendum) make UnmodifyableHRegionInfo deprecated


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

Branch: refs/heads/HBASE-18467
Commit: 81d2927e4cd31a40d091a2f77133cafb394949e6
Parents: a6d8ced
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Fri Sep 15 19:19:13 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Sat Sep 16 17:05:17 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java    | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/81d2927e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
index dfe04e7..128a77d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
@@ -22,7 +22,11 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 
-@InterfaceAudience.Private
+/**
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+ */
+@InterfaceAudience.Public
+@Deprecated
 class UnmodifyableHRegionInfo extends HRegionInfo {
   /*
    * Creates an unmodifyable copy of an HRegionInfo


[27/47] hbase git commit: HBASE-18808 Ineffective config check in BackupLogCleaner#getDeletableFiles()

Posted by bu...@apache.org.
HBASE-18808 Ineffective config check in BackupLogCleaner#getDeletableFiles()

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


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

Branch: refs/heads/HBASE-18467
Commit: 9f0863ce5f25276e22ac0d3b2c03b4f6f2296578
Parents: ff0c1d2
Author: Reid Chan <re...@outlook.com>
Authored: Mon Sep 18 14:51:09 2017 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Sep 18 10:16:14 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/backup/master/BackupLogCleaner.java | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9f0863ce/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
index 08a7eba..0f1722f 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
@@ -76,8 +76,10 @@ public class BackupLogCleaner extends BaseLogCleanerDelegate {
     // all members of this class are null if backup is disabled,
     // so we cannot filter the files
     if (this.getConf() == null || !BackupManager.isBackupEnabled(getConf())) {
-      LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY
-          + " setting");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Backup is not enabled. Check your "
+            + BackupRestoreConstants.BACKUP_ENABLE_KEY + " setting");
+      }
       return files;
     }
 
@@ -117,12 +119,11 @@ public class BackupLogCleaner extends BaseLogCleanerDelegate {
   @Override
   public void setConf(Configuration config) {
     // If backup is disabled, keep all members null
+    super.setConf(config);
     if (!config.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY,
       BackupRestoreConstants.BACKUP_ENABLE_DEFAULT)) {
       LOG.warn("Backup is disabled - allowing all wals to be deleted");
-      return;
     }
-    super.setConf(config);
   }
 
   @Override


[25/47] hbase git commit: HBASE-18836 Note need for explicit javax.el and exclude from shaded artifacts

Posted by bu...@apache.org.
HBASE-18836 Note need for explicit javax.el and exclude from shaded artifacts

Signed-off-by: Huaxiang Sun <hu...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 5264f04b6dd0c62dbde1a3af93b17f468dfc3998
Parents: fc13cf7
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Sep 18 08:47:22 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Sep 18 09:52:10 2017 -0500

----------------------------------------------------------------------
 hbase-rest/pom.xml                          | 3 +++
 hbase-server/pom.xml                        | 3 +++
 hbase-shaded/hbase-shaded-mapreduce/pom.xml | 4 ++++
 hbase-thrift/pom.xml                        | 3 +++
 4 files changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5264f04b/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 4553293..0b18809 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -308,6 +308,9 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <!-- Specifically needed for jetty-jsp, included
+         to bypass version scanning that hits a bad repo
+         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/5264f04b/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 03891a1..ad080f1 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -495,6 +495,9 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <!-- Specifically needed for jetty-jsp, included
+         to bypass version scanning that hits a bad repo
+         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/5264f04b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-mapreduce/pom.xml b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
index d42735a..2afa3c2 100644
--- a/hbase-shaded/hbase-shaded-mapreduce/pom.xml
+++ b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
@@ -130,6 +130,10 @@
                 <artifactId>jetty-jsp</artifactId>
               </exclusion>
               <exclusion>
+                <groupId>org.glassfish</groupId>
+                <artifactId>javax.el</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>org.eclipse.jetty</groupId>
                 <artifactId>jetty-webapp</artifactId>
               </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/5264f04b/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index 36c992d..0dc0dde 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -299,6 +299,9 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <!-- Specifically needed for jetty-jsp, included
+         to bypass version scanning that hits a bad repo
+         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>


[34/47] hbase git commit: HBASE-18852 Take down the hbasecon asia banner on home page

Posted by bu...@apache.org.
HBASE-18852 Take down the hbasecon asia banner on home page


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

Branch: refs/heads/HBASE-18467
Commit: bc790fe26acb9449afd7b3d316e6661933a86b40
Parents: 58f9cd3
Author: Michael Stack <st...@apache.org>
Authored: Tue Sep 19 17:00:01 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Tue Sep 19 17:00:01 2017 -0700

----------------------------------------------------------------------
 src/site/site.xml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bc790fe2/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index e038f91..400ae8c 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -43,9 +43,9 @@
     </fluidoSkin>
   </custom>
   <bannerLeft>
-    <name>HBaseCon2017</name>
-    <src>images/hbaseconasia2017.png</src>
-    <href>https://www.eventbrite.com/e/hbasecon-asia-2017-tickets-34935546159</href>
+    <name />
+    <src />
+    <href />
     <!--
     <name/>
     <height>0</height>


[32/47] hbase git commit: HBASE-18796 Admin#isTableAvailable returns incorrect result before daughter regions are opened

Posted by bu...@apache.org.
HBASE-18796 Admin#isTableAvailable returns incorrect result before daughter regions are opened

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 29a3ff303799299ce42b57e85b2a2ac575dab474
Parents: a29ea36
Author: Abhishek Singh Chouhan <ac...@apache.org>
Authored: Mon Sep 18 15:02:11 2017 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Sep 19 15:06:15 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 13 ++++++-
 .../hadoop/hbase/TestMetaTableAccessor.java     | 41 ++++++++++++++++++++
 2 files changed, 52 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/29a3ff30/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 1b81359..47fffa2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1724,8 +1724,8 @@ public class MetaTableAccessor {
       Put putA = makePutFromRegionInfo(splitA);
       Put putB = makePutFromRegionInfo(splitB);
 
-      addLocation(putA, sn, 1, -1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
-      addLocation(putB, sn, 1, -1, splitB.getReplicaId());
+      addSequenceNum(putA, 1, -1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
+      addSequenceNum(putB, 1, -1, splitB.getReplicaId());
 
       // Add empty locations for region replicas of daughters so that number of replicas can be
       // cached whenever the primary region is looked up from meta
@@ -2100,6 +2100,15 @@ public class MetaTableAccessor {
     return p.getClass().getSimpleName() + p.toJSON();
   }
 
+  public static Put addSequenceNum(final Put p, long openSeqNum, long time, int replicaId) {
+    if (time <= 0) {
+      time = EnvironmentEdgeManager.currentTime();
+    }
+    p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), time,
+      Bytes.toBytes(openSeqNum));
+    return p;
+  }
+
   /**
    * Get replication position for a peer in a region.
    * @param connection connection we're using

http://git-wip-us.apache.org/repos/asf/hbase/blob/29a3ff30/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index 6e9454f..a0cd236 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -699,5 +699,46 @@ public class TestMetaTableAccessor {
       assertTrue(prevCalls < scheduler.numPriorityCalls);
     }
   }
+
+  @Test
+  public void testEmptyMetaDaughterLocationDuringSplit() throws IOException {
+    long regionId = System.currentTimeMillis();
+    ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
+    HRegionInfo parent = new HRegionInfo(TableName.valueOf("table_foo"), HConstants.EMPTY_START_ROW,
+        HConstants.EMPTY_END_ROW, false, regionId, 0);
+    HRegionInfo splitA = new HRegionInfo(TableName.valueOf("table_foo"), HConstants.EMPTY_START_ROW,
+        Bytes.toBytes("a"), false, regionId + 1, 0);
+    HRegionInfo splitB = new HRegionInfo(TableName.valueOf("table_foo"), Bytes.toBytes("a"),
+        HConstants.EMPTY_END_ROW, false, regionId + 1, 0);
+
+    Table meta = MetaTableAccessor.getMetaHTable(connection);
+    try {
+      List<HRegionInfo> regionInfos = Lists.newArrayList(parent);
+      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
+
+      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3, false);
+      Get get1 = new Get(splitA.getRegionName());
+      Result resultA = meta.get(get1);
+      Cell serverCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        MetaTableAccessor.getServerColumn(splitA.getReplicaId()));
+      Cell startCodeCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        MetaTableAccessor.getStartCodeColumn(splitA.getReplicaId()));
+      assertNull(serverCellA);
+      assertNull(startCodeCellA);
+
+      Get get2 = new Get(splitA.getRegionName());
+      Result resultB = meta.get(get2);
+      Cell serverCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        MetaTableAccessor.getServerColumn(splitB.getReplicaId()));
+      Cell startCodeCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        MetaTableAccessor.getStartCodeColumn(splitB.getReplicaId()));
+      assertNull(serverCellB);
+      assertNull(startCodeCellB);
+    } finally {
+      if (meta != null) {
+        meta.close();
+      }
+    }
+  }
 }
 


[23/47] hbase git commit: HBASE-18834 fix shellcheck warning on hbase personality

Posted by bu...@apache.org.
HBASE-18834 fix shellcheck warning on hbase personality

Signed-off-by: Dima Spivak <di...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: efb95a17945cc91aa639396f6f6c528b52c71fcb
Parents: 2a8fb7a
Author: Sean Busbey <bu...@apache.org>
Authored: Sun Sep 17 23:54:37 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Sep 18 08:54:37 2017 -0500

----------------------------------------------------------------------
 dev-support/hbase-personality.sh | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/efb95a17/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index cf7c6b9..b7a2f97 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -119,8 +119,7 @@ function personality_modules
     yetus_debug "EXCLUDE_TESTS_URL = ${EXCLUDE_TESTS_URL}"
     yetus_debug "INCLUDE_TESTS_URL = ${INCLUDE_TESTS_URL}"
     if [[ -n "$EXCLUDE_TESTS_URL" ]]; then
-        wget "$EXCLUDE_TESTS_URL" -O "excludes"
-        if [[ $? -eq 0 ]]; then
+        if wget "$EXCLUDE_TESTS_URL" -O "excludes"; then
           excludes=$(cat excludes)
           yetus_debug "excludes=${excludes}"
           if [[ -n "${excludes}" ]]; then
@@ -132,8 +131,7 @@ function personality_modules
                "${EXCLUDE_TESTS_URL}. Ignoring and proceeding."
         fi
     elif [[ -n "$INCLUDE_TESTS_URL" ]]; then
-        wget "$INCLUDE_TESTS_URL" -O "includes"
-        if [[ $? -eq 0 ]]; then
+        if wget "$INCLUDE_TESTS_URL" -O "includes"; then
           includes=$(cat includes)
           yetus_debug "includes=${includes}"
           if [[ -n "${includes}" ]]; then


[42/47] hbase git commit: HBASE-18813 TestCanaryTool fails on branch-1 / branch-1.4

Posted by bu...@apache.org.
HBASE-18813 TestCanaryTool fails on branch-1 / branch-1.4

Disable units where argument matching is intermittently failing


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

Branch: refs/heads/HBASE-18467
Commit: 11d749527a926de44036154f675eb055190b4fad
Parents: 5f238b3
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Sep 15 10:24:05 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Sep 22 09:48:17 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/11d74952/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
index ba77035..190d5a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
@@ -119,6 +120,7 @@ public class TestCanaryTool {
   }
 
   @Test
+  @Ignore("Intermittent argument matching failures, see HBASE-18813")
   public void testReadTableTimeouts() throws Exception {
     final TableName [] tableNames = new TableName[2];
     tableNames[0] = TableName.valueOf(name.getMethodName() + "1");
@@ -162,6 +164,7 @@ public class TestCanaryTool {
   }
 
   @Test
+  @Ignore("Intermittent argument matching failures, see HBASE-18813")
   public void testWriteTableTimeout() throws Exception {
     ExecutorService executor = new ScheduledThreadPoolExecutor(1);
     Canary.RegionStdOutSink sink = spy(new Canary.RegionStdOutSink());


[44/47] hbase git commit: HBASE-18787 Fix the "dependencies connecting to an HBase cluster"

Posted by bu...@apache.org.
HBASE-18787 Fix the "dependencies connecting to an HBase cluster"


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

Branch: refs/heads/HBASE-18467
Commit: bb9f01c1e400e8594c6356fecff1922c629f9a1a
Parents: 21a61ad
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Mon Sep 11 16:30:32 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Sat Sep 23 01:56:38 2017 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/configuration.adoc | 19 +++++++------------
 1 file changed, 7 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bb9f01c1/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index 7e46156..6c550c4 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -547,19 +547,14 @@ Usually this ensemble location is kept out in the _hbase-site.xml_ and is picked
 
 If you are configuring an IDE to run an HBase client, you should include the _conf/_ directory on your classpath so _hbase-site.xml_ settings can be found (or add _src/test/resources_ to pick up the hbase-site.xml used by tests).
 
-Minimally, an HBase client needs several libraries in its `CLASSPATH` when connecting to a cluster, including:
-[source]
+Minimally, an HBase client needs hbase-client module in its dependencies when connecting to a cluster:
+[source,xml]
 ----
-
-commons-configuration (commons-configuration-1.6.jar)
-commons-lang3 (commons-lang3-3.6.jar)
-commons-logging (commons-logging-1.1.1.jar)
-hadoop-core (hadoop-core-1.0.0.jar)
-hbase (hbase-0.92.0.jar)
-log4j (log4j-1.2.16.jar)
-slf4j-api (slf4j-api-1.5.8.jar)
-slf4j-log4j (slf4j-log4j12-1.5.8.jar)
-zookeeper (zookeeper-3.4.2.jar)
+<dependency>
+  <groupId>org.apache.hbase</groupId>
+  <artifactId>hbase-client</artifactId>
+  <version>1.2.4</version>
+</dependency>
 ----
 
 A basic example _hbase-site.xml_ for client only may look as follows:


[16/47] hbase git commit: HBASE-14004 [Replication] Inconsistency between Memstore and WAL may result in data in remote cluster that is not in the origin

Posted by bu...@apache.org.
HBASE-14004 [Replication] Inconsistency between Memstore and WAL may result in data in remote cluster that is not in the origin


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

Branch: refs/heads/HBASE-18467
Commit: 4341c3f554cf85e73d3bb536bdda33a83f463f16
Parents: f7a986c
Author: zhangduo <zh...@apache.org>
Authored: Thu Sep 14 17:26:36 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Sep 15 19:22:00 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/Threads.java   |  12 +-
 .../hbase/regionserver/HRegionServer.java       |  71 +++++-----
 .../hbase/regionserver/ReplicationService.java  |  17 ++-
 .../hbase/regionserver/wal/AbstractFSWAL.java   |  35 ++++-
 .../hbase/regionserver/wal/AsyncFSWAL.java      |   6 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  12 +-
 .../RecoveredReplicationSource.java             |  11 +-
 .../replication/regionserver/Replication.java   |  15 ++-
 .../regionserver/ReplicationSource.java         |  22 ++--
 .../ReplicationSourceInterface.java             |  14 +-
 .../regionserver/ReplicationSourceManager.java  |  71 +++++-----
 .../ReplicationSourceWALReader.java             |  25 ++--
 .../regionserver/WALEntryStream.java            | 130 ++++++++-----------
 .../regionserver/WALFileLengthProvider.java     |  34 +++++
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |   2 +-
 .../hadoop/hbase/wal/DisabledWALProvider.java   |   8 +-
 .../hbase/wal/RegionGroupingProvider.java       |   2 +-
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |   6 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  11 +-
 .../apache/hadoop/hbase/wal/WALProvider.java    |  12 +-
 .../replication/ReplicationSourceDummy.java     |  13 +-
 .../replication/TestReplicationSource.java      |   5 +-
 .../TestReplicationSourceManager.java           |  10 +-
 .../regionserver/TestWALEntryStream.java        |  87 ++++++++-----
 .../apache/hadoop/hbase/wal/IOTestProvider.java |   2 +-
 25 files changed, 365 insertions(+), 268 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
index 35bf2b7..b39a5e8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
@@ -59,7 +59,7 @@ public class Threads {
    * @param t thread to run
    * @return Returns the passed Thread <code>t</code>.
    */
-  public static Thread setDaemonThreadRunning(final Thread t) {
+  public static <T extends Thread> T setDaemonThreadRunning(T t) {
     return setDaemonThreadRunning(t, t.getName());
   }
 
@@ -69,8 +69,7 @@ public class Threads {
    * @param name new name
    * @return Returns the passed Thread <code>t</code>.
    */
-  public static Thread setDaemonThreadRunning(final Thread t,
-    final String name) {
+  public static <T extends Thread> T setDaemonThreadRunning(T t, String name) {
     return setDaemonThreadRunning(t, name, null);
   }
 
@@ -78,12 +77,11 @@ public class Threads {
    * Utility method that sets name, daemon status and starts passed thread.
    * @param t thread to frob
    * @param name new name
-   * @param handler A handler to set on the thread.  Pass null if want to
-   * use default handler.
+   * @param handler A handler to set on the thread. Pass null if want to use default handler.
    * @return Returns the passed Thread <code>t</code>.
    */
-  public static Thread setDaemonThreadRunning(final Thread t,
-    final String name, final UncaughtExceptionHandler handler) {
+  public static <T extends Thread> T setDaemonThreadRunning(T t, String name,
+      UncaughtExceptionHandler handler) {
     t.setName(name);
     if (handler != null) {
       t.setUncaughtExceptionHandler(handler);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/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 62987c0..f648c2f 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
@@ -1586,7 +1586,7 @@ public class HRegionServer extends HasThread implements
       // Save it in a file, this will allow to see if we crash
       ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
 
-      this.walFactory = setupWALAndReplication();
+      setupWALAndReplication();
       // Init in here rather than in constructor after thread name has been set
       this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this));
       this.metricsTable = new MetricsTable(new MetricsTableWrapperAggregateImpl(this));
@@ -1855,13 +1855,12 @@ public class HRegionServer extends HasThread implements
   /**
    * Setup WAL log and replication if enabled.
    * Replication setup is done in here because it wants to be hooked up to WAL.
-   * @return A WAL instance.
    * @throws IOException
    */
-  private WALFactory setupWALAndReplication() throws IOException {
+  private void setupWALAndReplication() throws IOException {
     // TODO Replication make assumptions here based on the default filesystem impl
-    final Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    final String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString());
+    Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString());
 
     Path logDir = new Path(walRootDir, logName);
     if (LOG.isDebugEnabled()) LOG.debug("logDir=" + logDir);
@@ -1875,7 +1874,7 @@ public class HRegionServer extends HasThread implements
     createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir);
 
     // listeners the wal factory will add to wals it creates.
-    final List<WALActionsListener> listeners = new ArrayList<>();
+    List<WALActionsListener> listeners = new ArrayList<>();
     listeners.add(new MetricsWAL());
     if (this.replicationSourceHandler != null &&
         this.replicationSourceHandler.getWALActionsListener() != null) {
@@ -1883,7 +1882,21 @@ public class HRegionServer extends HasThread implements
       listeners.add(this.replicationSourceHandler.getWALActionsListener());
     }
 
-    return new WALFactory(conf, listeners, serverName.toString());
+    // There is a cyclic dependency between ReplicationSourceHandler and WALFactory.
+    // We use WALActionsListener to get the newly rolled WALs, so we need to get the
+    // WALActionsListeners from ReplicationSourceHandler before constructing WALFactory. And then
+    // ReplicationSourceHandler need to use WALFactory get the length of the wal file being written.
+    // So we here we need to construct WALFactory first, and then pass it to the initialize method
+    // of ReplicationSourceHandler.
+    WALFactory factory = new WALFactory(conf, listeners, serverName.toString());
+    this.walFactory = factory;
+    if (this.replicationSourceHandler != null) {
+      this.replicationSourceHandler.initialize(this, walFs, logDir, oldLogDir, factory);
+    }
+    if (this.replicationSinkHandler != null &&
+        this.replicationSinkHandler != this.replicationSourceHandler) {
+      this.replicationSinkHandler.initialize(this, walFs, logDir, oldLogDir, factory);
+    }
   }
 
   public MetricsRegionServer getRegionServerMetrics() {
@@ -2898,7 +2911,7 @@ public class HRegionServer extends HasThread implements
   /**
    * Load the replication service objects, if any
    */
-  static private void createNewReplicationInstance(Configuration conf,
+  private static void createNewReplicationInstance(Configuration conf,
     HRegionServer server, FileSystem walFs, Path walDir, Path oldWALDir) throws IOException{
 
     if ((server instanceof HMaster) && (!LoadBalancer.isTablesOnMaster(conf) ||
@@ -2908,47 +2921,41 @@ public class HRegionServer extends HasThread implements
 
     // read in the name of the source replication class from the config file.
     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
-                               HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
+      HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
 
     // read in the name of the sink replication class from the config file.
     String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
-                             HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
+      HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
 
     // If both the sink and the source class names are the same, then instantiate
     // only one object.
     if (sourceClassname.equals(sinkClassname)) {
-      server.replicationSourceHandler = (ReplicationSourceService)
-                                         newReplicationInstance(sourceClassname,
-                                         conf, server, walFs, walDir, oldWALDir);
-      server.replicationSinkHandler = (ReplicationSinkService)
-                                         server.replicationSourceHandler;
+      server.replicationSourceHandler =
+          (ReplicationSourceService) newReplicationInstance(sourceClassname, conf, server, walFs,
+            walDir, oldWALDir);
+      server.replicationSinkHandler = (ReplicationSinkService) server.replicationSourceHandler;
     } else {
-      server.replicationSourceHandler = (ReplicationSourceService)
-                                         newReplicationInstance(sourceClassname,
-                                         conf, server, walFs, walDir, oldWALDir);
-      server.replicationSinkHandler = (ReplicationSinkService)
-                                         newReplicationInstance(sinkClassname,
-                                         conf, server, walFs, walDir, oldWALDir);
+      server.replicationSourceHandler =
+          (ReplicationSourceService) newReplicationInstance(sourceClassname, conf, server, walFs,
+            walDir, oldWALDir);
+      server.replicationSinkHandler = (ReplicationSinkService) newReplicationInstance(sinkClassname,
+        conf, server, walFs, walDir, oldWALDir);
     }
   }
 
-  static private ReplicationService newReplicationInstance(String classname,
-    Configuration conf, HRegionServer server, FileSystem walFs, Path logDir,
-    Path oldLogDir) throws IOException{
-
-    Class<?> clazz = null;
+  private static ReplicationService newReplicationInstance(String classname, Configuration conf,
+      HRegionServer server, FileSystem walFs, Path logDir, Path oldLogDir) throws IOException {
+    Class<? extends ReplicationService> clazz = null;
     try {
       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-      clazz = Class.forName(classname, true, classLoader);
+      clazz = Class.forName(classname, true, classLoader).asSubclass(ReplicationService.class);
     } catch (java.lang.ClassNotFoundException nfe) {
       throw new IOException("Could not find class for " + classname);
     }
 
-    // create an instance of the replication object.
-    ReplicationService service = (ReplicationService)
-                              ReflectionUtils.newInstance(clazz, conf);
-    service.initialize(server, walFs, logDir, oldLogDir);
-    return service;
+    // create an instance of the replication object, but do not initialize it here as we need to use
+    // WALFactory when initializing.
+    return ReflectionUtils.newInstance(clazz, conf);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
index d88450a..f3bc188 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationService.java
@@ -20,17 +20,17 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Server;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
 
 /**
- * Gateway to Cluster Replication.
- * Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
- * One such application is a cross-datacenter
- * replication service that can keep two hbase clusters in sync.
+ * Gateway to Cluster Replication. Used by
+ * {@link org.apache.hadoop.hbase.regionserver.HRegionServer}. One such application is a
+ * cross-datacenter replication service that can keep two hbase clusters in sync.
  */
 @InterfaceAudience.Private
 public interface ReplicationService {
@@ -39,9 +39,8 @@ public interface ReplicationService {
    * Initializes the replication service object.
    * @throws IOException
    */
-  void initialize(
-    Server rs, FileSystem fs, Path logdir, Path oldLogDir
-  ) throws IOException;
+  void initialize(Server rs, FileSystem fs, Path logdir, Path oldLogDir,
+      WALFileLengthProvider walFileLengthProvider) throws IOException;
 
   /**
    * Start replication services.

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 8b99676..8157108 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -17,9 +17,12 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.*;
+import static org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
 
+import com.lmax.disruptor.RingBuffer;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.management.MemoryType;
@@ -29,6 +32,7 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
+import java.util.OptionalLong;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -58,6 +62,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.util.DrainBarrier;
@@ -68,6 +73,7 @@ import org.apache.hadoop.hbase.wal.WAL;
 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.WALProvider.WriterBase;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.NullScope;
@@ -75,9 +81,6 @@ import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import com.lmax.disruptor.RingBuffer;
-
 /**
  * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
  * WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
@@ -105,7 +108,7 @@ import com.lmax.disruptor.RingBuffer;
  * (Need to keep our own file lengths, not rely on HDFS).
  */
 @InterfaceAudience.Private
-public abstract class AbstractFSWAL<W> implements WAL {
+public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
 
   private static final Log LOG = LogFactory.getLog(AbstractFSWAL.class);
 
@@ -984,6 +987,28 @@ public abstract class AbstractFSWAL<W> implements WAL {
   }
 
   /**
+   * if the given {@code path} is being written currently, then return its length.
+   * <p>
+   * This is used by replication to prevent replicating unacked log entries. See
+   * https://issues.apache.org/jira/browse/HBASE-14004 for more details.
+   */
+  @Override
+  public OptionalLong getLogFileSizeIfBeingWritten(Path path) {
+    rollWriterLock.lock();
+    try {
+      Path currentPath = getOldPath();
+      if (path.equals(currentPath)) {
+        W writer = this.writer;
+        return writer != null ? OptionalLong.of(writer.getLength()) : OptionalLong.empty();
+      } else {
+        return OptionalLong.empty();
+      }
+    } finally {
+      rollWriterLock.unlock();
+    }
+  }
+
+  /**
    * NOTE: This append, at a time that is usually after this call returns, starts an mvcc
    * transaction by calling 'begin' wherein which we assign this update a sequenceid. At assignment
    * time, we stamp all the passed in Cells inside WALEdit with their sequenceId. You must

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 7e91f8c..42183ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -707,8 +707,10 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   @Override
   protected void doShutdown() throws IOException {
     waitForSafePoint();
-    this.writer.close();
-    this.writer = null;
+    if (this.writer != null) {
+      this.writer.close();
+      this.writer = null;
+    }
     closeExecutor.shutdown();
     IOException error = new IOException("WAL has been closed");
     syncFutures.forEach(f -> f.done(f.getTxid(), error));

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 7298137..7e0fc37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import com.lmax.disruptor.BlockingWaitStrategy;
 import com.lmax.disruptor.EventHandler;
 import com.lmax.disruptor.ExceptionHandler;
@@ -46,8 +45,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.*;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HasThread;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.FSHLogProvider;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -62,6 +65,9 @@ import org.apache.htrace.NullScope;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * The default implementation of FSWAL.

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index 3594868..248a52a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -50,13 +50,12 @@ public class RecoveredReplicationSource extends ReplicationSource {
   private String actualPeerId;
 
   @Override
-  public void init(final Configuration conf, final FileSystem fs,
-      final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
-      final ReplicationPeers replicationPeers, final Stoppable stopper,
-      final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
-      final MetricsSource metrics) throws IOException {
+  public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
+      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Stoppable stopper,
+      String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
+      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     super.init(conf, fs, manager, replicationQueues, replicationPeers, stopper, peerClusterZnode,
-      clusterId, replicationEndpoint, metrics);
+      clusterId, replicationEndpoint, walFileLengthProvider, metrics);
     this.actualPeerId = this.replicationQueueInfo.getPeerId();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 9fd1a87..d26f253 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.OptionalLong;
 import java.util.UUID;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -85,6 +86,7 @@ public class Replication extends WALActionsListener.Base implements
   private int statsThreadPeriod;
   // ReplicationLoad to access replication metrics
   private ReplicationLoad replicationLoad;
+
   /**
    * Instantiate the replication management (if rep is enabled).
    * @param server Hosting server
@@ -93,9 +95,8 @@ public class Replication extends WALActionsListener.Base implements
    * @param oldLogDir directory where logs are archived
    * @throws IOException
    */
-  public Replication(final Server server, final FileSystem fs,
-      final Path logDir, final Path oldLogDir) throws IOException{
-    initialize(server, fs, logDir, oldLogDir);
+  public Replication(Server server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException {
+    initialize(server, fs, logDir, oldLogDir, p -> OptionalLong.empty());
   }
 
   /**
@@ -104,8 +105,8 @@ public class Replication extends WALActionsListener.Base implements
   public Replication() {
   }
 
-  public void initialize(final Server server, final FileSystem fs,
-      final Path logDir, final Path oldLogDir) throws IOException {
+  public void initialize(Server server, FileSystem fs, Path logDir, Path oldLogDir,
+      WALFileLengthProvider walFileLengthProvider) throws IOException {
     this.server = server;
     this.conf = this.server.getConfiguration();
     this.replicationForBulkLoadData = isReplicationForBulkLoadDataEnabled(this.conf);
@@ -144,8 +145,8 @@ public class Replication extends WALActionsListener.Base implements
       throw new IOException("Could not read cluster id", ke);
     }
     this.replicationManager =
-        new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker,
-            conf, this.server, fs, logDir, oldLogDir, clusterId);
+        new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker, conf,
+            this.server, fs, logDir, oldLogDir, clusterId, walFileLengthProvider);
     this.statsThreadPeriod =
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
     LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 6c96852..d16a68f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -62,6 +60,8 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 
 /**
  * Class that handles the source of a replication stream.
@@ -73,7 +73,6 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
  * A stream is considered down when we cannot contact a region server on the
  * peer cluster for more than 55 seconds by default.
  * </p>
- *
  */
 @InterfaceAudience.Private
 public class ReplicationSource extends Thread implements ReplicationSourceInterface {
@@ -123,6 +122,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   private ReplicationThrottler throttler;
   private long defaultBandwidth;
   private long currentBandwidth;
+  private WALFileLengthProvider walFileLengthProvider;
   protected final ConcurrentHashMap<String, ReplicationSourceShipper> workerThreads =
       new ConcurrentHashMap<>();
 
@@ -147,12 +147,10 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
    * @throws IOException
    */
   @Override
-  public void init(final Configuration conf, final FileSystem fs,
-      final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
-      final ReplicationPeers replicationPeers, final Stoppable stopper,
-      final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
-      final MetricsSource metrics)
-          throws IOException {
+  public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
+      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Stoppable stopper,
+      String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
+      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.stopper = stopper;
     this.conf = HBaseConfiguration.create(conf);
     this.waitOnEndpointSeconds =
@@ -181,6 +179,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     currentBandwidth = getCurrentBandwidth();
     this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0);
     this.totalBufferUsed = manager.getTotalBufferUsed();
+    this.walFileLengthProvider = walFileLengthProvider;
     LOG.info("peerClusterZnode=" + peerClusterZnode + ", ReplicationSource : " + peerId
         + ", currentBandwidth=" + this.currentBandwidth);
   }
@@ -560,4 +559,9 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     totalReplicatedEdits.addAndGet(entries.size());
     totalBufferUsed.addAndGet(-batchSize);
   }
+
+  @Override
+  public WALFileLengthProvider getWALFileLengthProvider() {
+    return walFileLengthProvider;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index da89aba..066f799 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -53,11 +53,10 @@ public interface ReplicationSourceInterface {
    * @param clusterId
    * @throws IOException
    */
-  public void init(final Configuration conf, final FileSystem fs,
-      final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
-      final ReplicationPeers replicationPeers, final Stoppable stopper,
-      final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
-      final MetricsSource metrics) throws IOException;
+  void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
+      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Stoppable stopper,
+      String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
+      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException;
 
   /**
    * Add a log to the list of logs to replicate
@@ -147,6 +146,11 @@ public interface ReplicationSourceInterface {
   ReplicationSourceManager getSourceManager();
 
   /**
+   * @return the wal file length provider
+   */
+  WALFileLengthProvider getWALFileLengthProvider();
+
+  /**
    * Try to throttle when the peer config with a bandwidth
    * @param batchSize entries size will be pushed
    * @throws InterruptedException

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 5b54ce0..609274f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -19,9 +19,6 @@
 
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -31,7 +28,6 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -40,6 +36,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -68,10 +65,13 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 /**
  * This class is responsible to manage all the replication
  * sources. There are two classes of sources:
@@ -116,12 +116,12 @@ public class ReplicationSourceManager implements ReplicationListener {
   private final Path logDir;
   // Path to the wal archive
   private final Path oldLogDir;
+  private final WALFileLengthProvider walFileLengthProvider;
   // The number of ms that we wait before moving znodes, HBASE-3596
   private final long sleepBeforeFailover;
   // Homemade executer service for replication
   private final ThreadPoolExecutor executor;
 
-  private final Random rand;
   private final boolean replicationForBulkLoadDataEnabled;
 
   private Connection connection;
@@ -141,10 +141,10 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param oldLogDir the directory where old logs are archived
    * @param clusterId
    */
-  public ReplicationSourceManager(final ReplicationQueues replicationQueues,
-      final ReplicationPeers replicationPeers, final ReplicationTracker replicationTracker,
-      final Configuration conf, final Server server, final FileSystem fs, final Path logDir,
-      final Path oldLogDir, final UUID clusterId) throws IOException {
+  public ReplicationSourceManager(ReplicationQueues replicationQueues,
+      ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf,
+      Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId,
+      WALFileLengthProvider walFileLengthProvider) throws IOException {
     //CopyOnWriteArrayList is thread-safe.
     //Generally, reading is more than modifying.
     this.sources = new CopyOnWriteArrayList<>();
@@ -162,6 +162,7 @@ public class ReplicationSourceManager implements ReplicationListener {
     this.sleepBeforeFailover =
         conf.getLong("replication.sleep.before.failover", 30000); // 30 seconds
     this.clusterId = clusterId;
+    this.walFileLengthProvider = walFileLengthProvider;
     this.replicationTracker.registerListener(this);
     this.replicationPeers.getAllPeerIds();
     // It's preferable to failover 1 RS at a time, but with good zk servers
@@ -175,8 +176,7 @@ public class ReplicationSourceManager implements ReplicationListener {
     tfb.setNameFormat("ReplicationExecutor-%d");
     tfb.setDaemon(true);
     this.executor.setThreadFactory(tfb.build());
-    this.rand = new Random();
-    this.latestPaths = Collections.synchronizedSet(new HashSet<Path>());
+    this.latestPaths = new HashSet<Path>();
     replicationForBulkLoadDataEnabled =
         conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
           HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
@@ -243,7 +243,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * Adds a normal source per registered peer cluster and tries to process all
    * old region server wal queues
    */
-  protected void init() throws IOException, ReplicationException {
+  void init() throws IOException, ReplicationException {
     for (String id : this.replicationPeers.getConnectedPeerIds()) {
       addSource(id);
       if (replicationForBulkLoadDataEnabled) {
@@ -267,13 +267,13 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @return the source that was created
    * @throws IOException
    */
-  protected ReplicationSourceInterface addSource(String id) throws IOException,
-      ReplicationException {
+  @VisibleForTesting
+  ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException {
     ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
     ReplicationPeer peer = replicationPeers.getConnectedPeer(id);
-    ReplicationSourceInterface src =
-        getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
-          this.replicationPeers, server, id, this.clusterId, peerConfig, peer);
+    ReplicationSourceInterface src = getReplicationSource(this.conf, this.fs, this,
+      this.replicationQueues, this.replicationPeers, server, id, this.clusterId, peerConfig, peer,
+      walFileLengthProvider);
     synchronized (this.walsById) {
       this.sources.add(src);
       Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
@@ -330,7 +330,8 @@ public class ReplicationSourceManager implements ReplicationListener {
    * Get a copy of the wals of the first source on this rs
    * @return a sorted set of wal names
    */
-  protected Map<String, Map<String, SortedSet<String>>> getWALs() {
+  @VisibleForTesting
+  Map<String, Map<String, SortedSet<String>>> getWALs() {
     return Collections.unmodifiableMap(walsById);
   }
 
@@ -338,7 +339,8 @@ public class ReplicationSourceManager implements ReplicationListener {
    * Get a copy of the wals of the recovered sources on this rs
    * @return a sorted set of wal names
    */
-  protected Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
+  @VisibleForTesting
+  Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
     return Collections.unmodifiableMap(walsByIdRecoveredQueues);
   }
 
@@ -364,12 +366,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @return the normal source for the give peer if it exists, otherwise null.
    */
   public ReplicationSourceInterface getSource(String peerId) {
-    for (ReplicationSourceInterface source: getSources()) {
-      if (source.getPeerId().equals(peerId)) {
-        return source;
-      }
-    }
-    return null;
+    return getSources().stream().filter(s -> s.getPeerId().equals(peerId)).findFirst().orElse(null);
   }
 
   @VisibleForTesting
@@ -466,12 +463,11 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @return the created source
    * @throws IOException
    */
-  protected ReplicationSourceInterface getReplicationSource(final Configuration conf,
-      final FileSystem fs, final ReplicationSourceManager manager,
-      final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
-      final Server server, final String peerId, final UUID clusterId,
-      final ReplicationPeerConfig peerConfig, final ReplicationPeer replicationPeer)
-      throws IOException {
+  private ReplicationSourceInterface getReplicationSource(Configuration conf, FileSystem fs,
+      ReplicationSourceManager manager, ReplicationQueues replicationQueues,
+      ReplicationPeers replicationPeers, Server server, String peerId, UUID clusterId,
+      ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer,
+      WALFileLengthProvider walFileLengthProvider) throws IOException {
     RegionServerCoprocessorHost rsServerHost = null;
     TableDescriptors tableDescriptors = null;
     if (server instanceof HRegionServer) {
@@ -507,8 +503,8 @@ public class ReplicationSourceManager implements ReplicationListener {
 
     MetricsSource metrics = new MetricsSource(peerId);
     // init replication source
-    src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId,
-      clusterId, replicationEndpoint, metrics);
+    src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId, clusterId,
+      replicationEndpoint, walFileLengthProvider, metrics);
 
     // init replication endpoint
     replicationEndpoint.init(new ReplicationEndpoint.Context(replicationPeer.getConfiguration(),
@@ -674,7 +670,8 @@ public class ReplicationSourceManager implements ReplicationListener {
       // Wait a bit before transferring the queues, we may be shutting down.
       // This sleep may not be enough in some cases.
       try {
-        Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
+        Thread.sleep(sleepBeforeFailover +
+            (long) (ThreadLocalRandom.current().nextFloat() * sleepBeforeFailover));
       } catch (InterruptedException e) {
         LOG.warn("Interrupted while waiting before transferring a queue.");
         Thread.currentThread().interrupt();
@@ -688,7 +685,7 @@ public class ReplicationSourceManager implements ReplicationListener {
       List<String> peers = rq.getUnClaimedQueueIds(rsZnode);
       while (peers != null && !peers.isEmpty()) {
         Pair<String, SortedSet<String>> peer = this.rq.claimQueue(rsZnode,
-            peers.get(rand.nextInt(peers.size())));
+          peers.get(ThreadLocalRandom.current().nextInt(peers.size())));
         long sleep = sleepBeforeFailover/2;
         if (peer != null) {
           newQueues.put(peer.getFirst(), peer.getSecond());
@@ -748,7 +745,7 @@ public class ReplicationSourceManager implements ReplicationListener {
           // enqueue sources
           ReplicationSourceInterface src =
               getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
-                server, peerId, this.clusterId, peerConfig, peer);
+                server, peerId, this.clusterId, peerConfig, peer, walFileLengthProvider);
           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
           // see removePeer
           synchronized (oldsources) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index bb5abe9..bb993c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -37,18 +37,18 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
-import org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.WALEntryStreamRuntimeException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 
 /**
  * Reads and filters WAL entries, groups the filtered entries into batches, and puts the batches onto a queue
@@ -127,8 +127,8 @@ public class ReplicationSourceWALReader extends Thread {
   public void run() {
     int sleepMultiplier = 1;
     while (isReaderRunning()) { // we only loop back here if something fatal happened to our stream
-      try (WALEntryStream entryStream =
-          new WALEntryStream(logQueue, fs, conf, currentPosition, source.getSourceMetrics())) {
+      try (WALEntryStream entryStream = new WALEntryStream(logQueue, fs, conf, currentPosition,
+          source.getWALFileLengthProvider(), source.getSourceMetrics())) {
         while (isReaderRunning()) { // loop here to keep reusing stream while we can
           if (!checkQuota()) {
             continue;
@@ -147,7 +147,7 @@ public class ReplicationSourceWALReader extends Thread {
           currentPosition = entryStream.getPosition();
           entryStream.reset(); // reuse stream
         }
-      } catch (IOException | WALEntryStreamRuntimeException e) { // stream related
+      } catch (IOException e) { // stream related
         if (sleepMultiplier < maxRetriesMultiplier) {
           LOG.debug("Failed to read stream of replication entries: " + e);
           sleepMultiplier++;
@@ -202,8 +202,9 @@ public class ReplicationSourceWALReader extends Thread {
   // if we get an EOF due to a zero-length log, and there are other logs in queue
   // (highly likely we've closed the current log), we've hit the max retries, and autorecovery is
   // enabled, then dump the log
-  private void handleEofException(Exception e) {
-    if (e.getCause() instanceof EOFException && logQueue.size() > 1 && this.eofAutoRecovery) {
+  private void handleEofException(IOException e) {
+    if (e instanceof EOFException ||
+        e.getCause() instanceof EOFException && logQueue.size() > 1 && this.eofAutoRecovery) {
       try {
         if (fs.getFileStatus(logQueue.peek()).getLen() == 0) {
           LOG.warn("Forcing removal of 0 length log in queue: " + logQueue.peek());

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
index 54511ae..3be4ca4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
@@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.NoSuchElementException;
+import java.util.OptionalLong;
 import java.util.concurrent.PriorityBlockingQueue;
 
 import org.apache.commons.logging.Log;
@@ -50,7 +50,7 @@ import org.apache.hadoop.ipc.RemoteException;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entry> {
+class WALEntryStream implements Closeable {
   private static final Log LOG = LogFactory.getLog(WALEntryStream.class);
 
   private Reader reader;
@@ -59,24 +59,11 @@ public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entr
   private Entry currentEntry;
   // position after reading current entry
   private long currentPosition = 0;
-  private PriorityBlockingQueue<Path> logQueue;
-  private FileSystem fs;
-  private Configuration conf;
-  private MetricsSource metrics;
-
-  /**
-   * Create an entry stream over the given queue
-   * @param logQueue the queue of WAL paths
-   * @param fs {@link FileSystem} to use to create {@link Reader} for this stream
-   * @param conf {@link Configuration} to use to create {@link Reader} for this stream
-   * @param metrics replication metrics
-   * @throws IOException
-   */
-  public WALEntryStream(PriorityBlockingQueue<Path> logQueue, FileSystem fs, Configuration conf,
-      MetricsSource metrics)
-      throws IOException {
-    this(logQueue, fs, conf, 0, metrics);
-  }
+  private final PriorityBlockingQueue<Path> logQueue;
+  private final FileSystem fs;
+  private final Configuration conf;
+  private final WALFileLengthProvider walFileLengthProvider;
+  private final MetricsSource metrics;
 
   /**
    * Create an entry stream over the given queue at the given start position
@@ -88,52 +75,41 @@ public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entr
    * @throws IOException
    */
   public WALEntryStream(PriorityBlockingQueue<Path> logQueue, FileSystem fs, Configuration conf,
-      long startPosition, MetricsSource metrics) throws IOException {
+      long startPosition, WALFileLengthProvider walFileLengthProvider, MetricsSource metrics)
+      throws IOException {
     this.logQueue = logQueue;
     this.fs = fs;
     this.conf = conf;
     this.currentPosition = startPosition;
+    this.walFileLengthProvider = walFileLengthProvider;
     this.metrics = metrics;
   }
 
   /**
    * @return true if there is another WAL {@link Entry}
-   * @throws WALEntryStreamRuntimeException if there was an Exception while reading
    */
-  @Override
-  public boolean hasNext() {
+  public boolean hasNext() throws IOException {
     if (currentEntry == null) {
-      try {
-        tryAdvanceEntry();
-      } catch (Exception e) {
-        throw new WALEntryStreamRuntimeException(e);
-      }
+      tryAdvanceEntry();
     }
     return currentEntry != null;
   }
 
   /**
    * @return the next WAL entry in this stream
-   * @throws WALEntryStreamRuntimeException if there was an IOException
+   * @throws IOException
    * @throws NoSuchElementException if no more entries in the stream.
    */
-  @Override
-  public Entry next() {
-    if (!hasNext()) throw new NoSuchElementException();
+  public Entry next() throws IOException {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
     Entry save = currentEntry;
     currentEntry = null; // gets reloaded by hasNext()
     return save;
   }
 
   /**
-   * Not supported.
-   */
-  @Override
-  public void remove() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
    * {@inheritDoc}
    */
   @Override
@@ -142,14 +118,6 @@ public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entr
   }
 
   /**
-   * @return the iterator over WAL entries in the queue.
-   */
-  @Override
-  public Iterator<Entry> iterator() {
-    return this;
-  }
-
-  /**
    * @return the position of the last Entry returned by next()
    */
   public long getPosition() {
@@ -195,24 +163,27 @@ public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entr
 
   private void tryAdvanceEntry() throws IOException {
     if (checkReader()) {
-      readNextEntryAndSetPosition();
-      if (currentEntry == null) { // no more entries in this log file - see if log was rolled
-        if (logQueue.size() > 1) { // log was rolled
-          // Before dequeueing, we should always get one more attempt at reading.
-          // This is in case more entries came in after we opened the reader,
-          // and a new log was enqueued while we were reading. See HBASE-6758
-          resetReader();
-          readNextEntryAndSetPosition();
-          if (currentEntry == null) {
-            if (checkAllBytesParsed()) { // now we're certain we're done with this log file
-              dequeueCurrentLog();
-              if (openNextLog()) {
-                readNextEntryAndSetPosition();
-              }
+      boolean beingWritten = readNextEntryAndSetPosition();
+      if (currentEntry == null && !beingWritten) {
+        // no more entries in this log file, and the file is already closed, i.e, rolled
+        // Before dequeueing, we should always get one more attempt at reading.
+        // This is in case more entries came in after we opened the reader, and the log is rolled
+        // while we were reading. See HBASE-6758
+        resetReader();
+        readNextEntryAndSetPosition();
+        if (currentEntry == null) {
+          if (checkAllBytesParsed()) { // now we're certain we're done with this log file
+            dequeueCurrentLog();
+            if (openNextLog()) {
+              readNextEntryAndSetPosition();
             }
           }
-        } // no other logs, we've simply hit the end of the current open log. Do nothing
+        }
       }
+      // if currentEntry != null then just return
+      // if currentEntry == null but the file is still being written, then we should not switch to
+      // the next log either, just return here and try next time to see if there are more entries in
+      // the current file
     }
     // do nothing if we don't have a WAL Reader (e.g. if there's no logs in queue)
   }
@@ -270,15 +241,30 @@ public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entr
     metrics.decrSizeOfLogQueue();
   }
 
-  private void readNextEntryAndSetPosition() throws IOException {
+  /**
+   * Returns whether the file is opened for writing.
+   */
+  private boolean readNextEntryAndSetPosition() throws IOException {
     Entry readEntry = reader.next();
     long readerPos = reader.getPosition();
+    OptionalLong fileLength = walFileLengthProvider.getLogFileSizeIfBeingWritten(currentPath);
+    if (fileLength.isPresent() && readerPos > fileLength.getAsLong()) {
+      // see HBASE-14004, for AsyncFSWAL which uses fan-out, it is possible that we read uncommitted
+      // data, so we need to make sure that we do not read beyond the committed file length.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("The provider tells us the valid length for " + currentPath + " is " +
+            fileLength.getAsLong() + ", but we have advanced to " + readerPos);
+      }
+      resetReader();
+      return true;
+    }
     if (readEntry != null) {
       metrics.incrLogEditsRead();
       metrics.incrLogReadInBytes(readerPos - currentPosition);
     }
     currentEntry = readEntry; // could be null
     setPosition(readerPos);
+    return fileLength.isPresent();
   }
 
   private void closeReader() throws IOException {
@@ -301,7 +287,9 @@ public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entr
     Path nextPath = logQueue.peek();
     if (nextPath != null) {
       openReader(nextPath);
-      if (reader != null) return true;
+      if (reader != null) {
+        return true;
+      }
     }
     return false;
   }
@@ -408,14 +396,4 @@ public class WALEntryStream implements Iterator<Entry>, Closeable, Iterable<Entr
     }
     return size;
   }
-
-  @InterfaceAudience.Private
-  public static class WALEntryStreamRuntimeException extends RuntimeException {
-    private static final long serialVersionUID = -6298201811259982568L;
-
-    public WALEntryStreamRuntimeException(Exception e) {
-      super(e);
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
new file mode 100644
index 0000000..010fa69
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.util.OptionalLong;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Used by replication to prevent replicating unacked log entries. See
+ * https://issues.apache.org/jira/browse/HBASE-14004 for more details.
+ */
+@InterfaceAudience.Private
+@FunctionalInterface
+public interface WALFileLengthProvider {
+
+  OptionalLong getLogFileSizeIfBeingWritten(Path path);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 9c80fe6..1a81b17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -115,7 +115,7 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   }
 
   @Override
-  public List<WAL> getWALs() throws IOException {
+  public List<WAL> getWALs() {
     if (wal == null) {
       return Collections.emptyList();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index c805ff3..a6d43d6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.OptionalLong;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -66,7 +67,7 @@ class DisabledWALProvider implements WALProvider {
   }
 
   @Override
-  public List<WAL> getWALs() throws IOException {
+  public List<WAL> getWALs() {
     List<WAL> wals = new ArrayList<>(1);
     wals.add(disabled);
     return wals;
@@ -232,6 +233,11 @@ class DisabledWALProvider implements WALProvider {
     public String toString() {
       return "WAL disabled.";
     }
+
+    @Override
+    public OptionalLong getLogFileSizeIfBeingWritten(Path path) {
+      return OptionalLong.empty();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
index 95b7dae..ab3a7d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
@@ -169,7 +169,7 @@ public class RegionGroupingProvider implements WALProvider {
   }
 
   @Override
-  public List<WAL> getWALs() throws IOException {
+  public List<WAL> getWALs() {
     List<WAL> wals = new ArrayList<>();
     for (WALProvider provider : cached.values()) {
       wals.addAll(provider.getWALs());

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index eede937..9ec58ab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -19,8 +19,6 @@
 
 package org.apache.hadoop.hbase.wal;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Map;
@@ -35,6 +33,8 @@ import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
+import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public interface WAL extends Closeable {
+public interface WAL extends Closeable, WALFileLengthProvider {
 
   /**
    * Registers WALActionsListener

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index aaa828f..efb8e2d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Collections;
 import java.util.List;
+import java.util.OptionalLong;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.logging.Log;
@@ -38,6 +39,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -63,7 +65,7 @@ import org.apache.hadoop.hbase.wal.WALProvider.Writer;
  * Alternatively, you may provide a custom implementation of {@link WALProvider} by class name.
  */
 @InterfaceAudience.Private
-public class WALFactory {
+public class WALFactory implements WALFileLengthProvider {
 
   private static final Log LOG = LogFactory.getLog(WALFactory.class);
 
@@ -230,7 +232,7 @@ public class WALFactory {
     }
   }
 
-  public List<WAL> getWALs() throws IOException {
+  public List<WAL> getWALs() {
     return provider.getWALs();
   }
 
@@ -450,4 +452,9 @@ public class WALFactory {
   public final WALProvider getMetaWALProvider() {
     return this.metaProvider.get();
   }
+
+  @Override
+  public OptionalLong getLogFileSizeIfBeingWritten(Path path) {
+    return getWALs().stream().map(w -> w.getLogFileSizeIfBeingWritten(path)).filter(o -> o.isPresent()).findAny().orElse(OptionalLong.empty());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
index ffcfcd4..c38f419 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
@@ -60,7 +60,7 @@ public interface WALProvider {
 
   /** @return the List of WALs that are used by this server
    */
-  List<WAL> getWALs() throws IOException;
+  List<WAL> getWALs();
 
   /**
    * persist outstanding WALs to storage and stop accepting new appends.
@@ -76,18 +76,20 @@ public interface WALProvider {
    */
   void close() throws IOException;
 
+  interface WriterBase extends Closeable {
+    long getLength();
+  }
+
   // Writers are used internally. Users outside of the WAL should be relying on the
   // interface provided by WAL.
-  interface Writer extends Closeable {
+  interface Writer extends WriterBase {
     void sync() throws IOException;
     void append(WAL.Entry entry) throws IOException;
-    long getLength();
   }
 
-  interface AsyncWriter extends Closeable {
+  interface AsyncWriter extends WriterBase {
     CompletableFuture<Long> sync();
     void append(WAL.Entry entry);
-    long getLength();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index e23e15b..bfe17b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
+import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
@@ -42,16 +43,17 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   String peerClusterId;
   Path currentPath;
   MetricsSource metrics;
+  WALFileLengthProvider walFileLengthProvider;
 
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
       ReplicationQueues rq, ReplicationPeers rp, Stoppable stopper, String peerClusterId,
-      UUID clusterId, ReplicationEndpoint replicationEndpoint, MetricsSource metrics)
-          throws IOException {
-
+      UUID clusterId, ReplicationEndpoint replicationEndpoint,
+      WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.manager = manager;
     this.peerClusterId = peerClusterId;
     this.metrics = metrics;
+    this.walFileLengthProvider = walFileLengthProvider;
   }
 
   @Override
@@ -135,4 +137,9 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   @Override
   public void postShipEdits(List<Entry> entries, int batchSize) {
   }
+
+  @Override
+  public WALFileLengthProvider getWALFileLengthProvider() {
+    return walFileLengthProvider;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
index 56a5bdc..ebb1bf8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
 import java.io.IOException;
+import java.util.OptionalLong;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -166,8 +167,8 @@ public class TestReplicationSource {
     testConf.setInt("replication.source.maxretriesmultiplier", 1);
     ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
     Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong());
-    source.init(testConf, null, manager, null, mockPeers, null, "testPeer",
-        null, replicationEndpoint, null);
+    source.init(testConf, null, manager, null, mockPeers, null, "testPeer", null,
+      replicationEndpoint, p -> OptionalLong.empty(), null);
     ExecutorService executor = Executors.newSingleThreadExecutor();
     Future<?> future = executor.submit(new Runnable() {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/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 9804df4..3934e05 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
@@ -61,9 +61,6 @@ import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -97,6 +94,9 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 
 /**
  * An abstract class that tests ReplicationSourceManager. Classes that extend this class should
@@ -646,8 +646,8 @@ public abstract class TestReplicationSourceManager {
     @Override
     public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
         ReplicationQueues rq, ReplicationPeers rp, Stoppable stopper, String peerClusterId,
-        UUID clusterId, ReplicationEndpoint replicationEndpoint, MetricsSource metrics)
-        throws IOException {
+        UUID clusterId, ReplicationEndpoint replicationEndpoint,
+        WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
       throw new IOException("Failing deliberately");
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index 5f3452a..d65054c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -31,6 +31,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.NoSuchElementException;
+import java.util.OptionalLong;
 import java.util.TreeMap;
 import java.util.concurrent.PriorityBlockingQueue;
 import java.util.concurrent.atomic.AtomicLong;
@@ -42,13 +43,10 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.WALEntryBatch;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -56,6 +54,7 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+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.hdfs.MiniDFSCluster;
@@ -67,11 +66,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.junit.runner.RunWith;
 import org.mockito.Mockito;
-import org.mockito.runners.MockitoJUnitRunner;
 
-@RunWith(MockitoJUnitRunner.class)
 @Category({ ReplicationTests.class, LargeTests.class })
 public class TestWALEntryStream {
 
@@ -84,8 +80,13 @@ public class TestWALEntryStream {
   private static final byte[] qualifier = Bytes.toBytes("qualifier");
   private static final HRegionInfo info =
       new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false);
-  private static final HTableDescriptor htd = new HTableDescriptor(tableName);
-  private static NavigableMap<byte[], Integer> scopes;
+  private static final NavigableMap<byte[], Integer> scopes = getScopes();
+
+  private static NavigableMap<byte[], Integer> getScopes() {
+    NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    scopes.put(family, 1);
+    return scopes;
+  }
 
   private WAL log;
   PriorityBlockingQueue<Path> walQueue;
@@ -103,10 +104,6 @@ public class TestWALEntryStream {
 
     cluster = TEST_UTIL.getDFSCluster();
     fs = cluster.getFileSystem();
-    scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
-    for (byte[] fam : htd.getFamiliesKeys()) {
-      scopes.put(fam, 0);
-    }
   }
 
   @AfterClass
@@ -151,10 +148,10 @@ public class TestWALEntryStream {
           log.rollWriter();
 
           try (WALEntryStream entryStream =
-              new WALEntryStream(walQueue, fs, conf, new MetricsSource("1"))) {
+              new WALEntryStream(walQueue, fs, conf, 0, log, new MetricsSource("1"))) {
             int i = 0;
-            for (WAL.Entry e : entryStream) {
-              assertNotNull(e);
+            while (entryStream.hasNext()) {
+              assertNotNull(entryStream.next());
               i++;
             }
             assertEquals(nbRows, i);
@@ -176,10 +173,9 @@ public class TestWALEntryStream {
   @Test
   public void testAppendsWithRolls() throws Exception {
     appendToLog();
-
     long oldPos;
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, 0, log, new MetricsSource("1"))) {
       // There's one edit in the log, read it. Reading past it needs to throw exception
       assertTrue(entryStream.hasNext());
       WAL.Entry entry = entryStream.next();
@@ -196,8 +192,8 @@ public class TestWALEntryStream {
 
     appendToLog();
 
-    try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, oldPos, new MetricsSource("1"))) {
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, oldPos,
+        log, new MetricsSource("1"))) {
       // Read the newly added entry, make sure we made progress
       WAL.Entry entry = entryStream.next();
       assertNotEquals(oldPos, entryStream.getPosition());
@@ -210,8 +206,8 @@ public class TestWALEntryStream {
     log.rollWriter();
     appendToLog();
 
-    try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, oldPos, new MetricsSource("1"))) {
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, oldPos,
+        log, new MetricsSource("1"))) {
       WAL.Entry entry = entryStream.next();
       assertNotEquals(oldPos, entryStream.getPosition());
       assertNotNull(entry);
@@ -236,7 +232,7 @@ public class TestWALEntryStream {
     appendToLog("1");
     appendToLog("2");// 2
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, 0, log, new MetricsSource("1"))) {
       assertEquals("1", getRow(entryStream.next()));
 
       appendToLog("3"); // 3 - comes in after reader opened
@@ -261,7 +257,7 @@ public class TestWALEntryStream {
   public void testNewEntriesWhileStreaming() throws Exception {
     appendToLog("1");
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, 0, log, new MetricsSource("1"))) {
       entryStream.next(); // we've hit the end of the stream at this point
 
       // some new entries come in while we're streaming
@@ -284,7 +280,7 @@ public class TestWALEntryStream {
     long lastPosition = 0;
     appendToLog("1");
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, 0, log, new MetricsSource("1"))) {
       entryStream.next(); // we've hit the end of the stream at this point
       appendToLog("2");
       appendToLog("3");
@@ -292,7 +288,7 @@ public class TestWALEntryStream {
     }
     // next stream should picks up where we left off
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, lastPosition, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, lastPosition, log, new MetricsSource("1"))) {
       assertEquals("2", getRow(entryStream.next()));
       assertEquals("3", getRow(entryStream.next()));
       assertFalse(entryStream.hasNext()); // done
@@ -309,14 +305,14 @@ public class TestWALEntryStream {
     long lastPosition = 0;
     appendEntriesToLog(3);
     // read only one element
-    try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, lastPosition, new MetricsSource("1"))) {
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, lastPosition,
+        log, new MetricsSource("1"))) {
       entryStream.next();
       lastPosition = entryStream.getPosition();
     }
     // there should still be two more entries from where we left off
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, lastPosition, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, lastPosition, log, new MetricsSource("1"))) {
       assertNotNull(entryStream.next());
       assertNotNull(entryStream.next());
       assertFalse(entryStream.hasNext());
@@ -327,7 +323,7 @@ public class TestWALEntryStream {
   @Test
   public void testEmptyStream() throws Exception {
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, 0, log, new MetricsSource("1"))) {
       assertFalse(entryStream.hasNext());
     }
   }
@@ -338,7 +334,7 @@ public class TestWALEntryStream {
     // get ending position
     long position;
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, conf, 0, log, new MetricsSource("1"))) {
       entryStream.next();
       entryStream.next();
       entryStream.next();
@@ -351,6 +347,7 @@ public class TestWALEntryStream {
     ReplicationSource source = Mockito.mock(ReplicationSource.class);
     when(source.getSourceManager()).thenReturn(mockSourceManager);
     when(source.getSourceMetrics()).thenReturn(new MetricsSource("1"));
+    when(source.getWALFileLengthProvider()).thenReturn(log);
     ReplicationSourceWALReader batcher = new ReplicationSourceWALReader(fs, conf,
         walQueue, 0, getDummyFilter(), source);
     Path walPath = walQueue.peek();
@@ -425,10 +422,6 @@ public class TestWALEntryStream {
     };
   }
 
-  private ReplicationQueueInfo getQueueInfo() {
-    return new ReplicationQueueInfo("1");
-  }
-
   class PathWatcher extends WALActionsListener.Base {
 
     Path currentPath;
@@ -440,4 +433,30 @@ public class TestWALEntryStream {
     }
   }
 
+  @Test
+  public void testReadBeyondCommittedLength() throws IOException, InterruptedException {
+    appendToLog("1");
+    appendToLog("2");
+    long size = log.getLogFileSizeIfBeingWritten(walQueue.peek()).getAsLong();
+    AtomicLong fileLength = new AtomicLong(size - 1);
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, 0,
+        p -> OptionalLong.of(fileLength.get()), new MetricsSource("1"))) {
+      assertTrue(entryStream.hasNext());
+      assertNotNull(entryStream.next());
+      // can not get log 2
+      assertFalse(entryStream.hasNext());
+      Thread.sleep(1000);
+      entryStream.reset();
+      // still can not get log 2
+      assertFalse(entryStream.hasNext());
+
+      // can get log 2 now
+      fileLength.set(size);
+      entryStream.reset();
+      assertTrue(entryStream.hasNext());
+      assertNotNull(entryStream.next());
+
+      assertFalse(entryStream.hasNext());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4341c3f5/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index 041d8ae..944a4f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -107,7 +107,7 @@ public class IOTestProvider implements WALProvider {
   }
 
   @Override
-  public List<WAL> getWALs() throws IOException {
+  public List<WAL> getWALs() {
     List<WAL> wals = new ArrayList<>(1);
     wals.add(log);
     return wals;


[22/47] hbase git commit: Revert "HBASE-18836 Note need for explicit javax.el and exclude from shaded artifacts"

Posted by bu...@apache.org.
Revert "HBASE-18836 Note need for explicit javax.el and exclude from shaded artifacts"

This reverts commit de32b11fe4536d07ac4ffee7c59152cebe4f150e.

Mistaken push of wrong branch.


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

Branch: refs/heads/HBASE-18467
Commit: 2a8fb7a9ae6ad88f6b134422b3ba481815e4edde
Parents: de32b11
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Sep 18 08:53:42 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Sep 18 08:53:42 2017 -0500

----------------------------------------------------------------------
 hbase-rest/pom.xml                          | 3 ---
 hbase-server/pom.xml                        | 3 ---
 hbase-shaded/hbase-shaded-mapreduce/pom.xml | 4 ----
 hbase-thrift/pom.xml                        | 3 ---
 4 files changed, 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2a8fb7a9/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 0b18809..4553293 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -308,9 +308,6 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
-    <!-- Specifically needed for jetty-jsp, included
-         to bypass version scanning that hits a bad repo
-         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a8fb7a9/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index ad080f1..03891a1 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -495,9 +495,6 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
-    <!-- Specifically needed for jetty-jsp, included
-         to bypass version scanning that hits a bad repo
-         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a8fb7a9/hbase-shaded/hbase-shaded-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-mapreduce/pom.xml b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
index 2afa3c2..d42735a 100644
--- a/hbase-shaded/hbase-shaded-mapreduce/pom.xml
+++ b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
@@ -130,10 +130,6 @@
                 <artifactId>jetty-jsp</artifactId>
               </exclusion>
               <exclusion>
-                <groupId>org.glassfish</groupId>
-                <artifactId>javax.el</artifactId>
-              </exclusion>
-              <exclusion>
                 <groupId>org.eclipse.jetty</groupId>
                 <artifactId>jetty-webapp</artifactId>
               </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2a8fb7a9/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index 0dc0dde..36c992d 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -299,9 +299,6 @@
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
-    <!-- Specifically needed for jetty-jsp, included
-         to bypass version scanning that hits a bad repo
-         see HBASE-18831 -->
     <dependency>
       <groupId>org.glassfish</groupId>
       <artifactId>javax.el</artifactId>


[08/47] hbase git commit: HBASE-18818 TestConnectionImplemenation fails

Posted by bu...@apache.org.
HBASE-18818 TestConnectionImplemenation fails


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

Branch: refs/heads/HBASE-18467
Commit: 98f417110976dc137f273defee7e8f163a471b30
Parents: 778f507
Author: Michael Stack <st...@apache.org>
Authored: Thu Sep 14 12:27:37 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 14 12:27:37 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/98f41711/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
index af7b652..1b3366b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
@@ -67,7 +67,7 @@ public class TestConnectionImplementation {
     // test that we fail to get a client to an unresolvable hostname, which
     // means it won't be cached
     ServerName badHost =
-        ServerName.valueOf("unknownhost.example.com:" + HConstants.DEFAULT_MASTER_PORT,
+        ServerName.valueOf("unknownhost.invalid:" + HConstants.DEFAULT_MASTER_PORT,
         System.currentTimeMillis());
     conn.getAdmin(badHost);
     fail("Obtaining admin to unresolvable hostname should have failed");
@@ -86,7 +86,7 @@ public class TestConnectionImplementation {
     // test that we fail to get a client to an unresolvable hostname, which
     // means it won't be cached
     ServerName badHost =
-        ServerName.valueOf("unknownhost.example.com:" + HConstants.DEFAULT_REGIONSERVER_PORT,
+        ServerName.valueOf("unknownhost.invalid:" + HConstants.DEFAULT_REGIONSERVER_PORT,
         System.currentTimeMillis());
     conn.getAdmin(badHost);
     fail("Obtaining client to unresolvable hostname should have failed");


[41/47] hbase git commit: HBASE-18651 Let ChaosMonkeyRunner expose the chaos monkey runner it creates

Posted by bu...@apache.org.
HBASE-18651 Let ChaosMonkeyRunner expose the chaos monkey runner it creates

Signed-off-by: Mike Drob <md...@apache.org>
Signed-off-by: Ted Yu <te...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 5f238b3ef46e7d5f31a9fb2dfff0111085af835d
Parents: e393599
Author: Reid Chan <re...@outlook.com>
Authored: Thu Sep 21 11:30:14 2017 +0800
Committer: Mike Drob <md...@apache.org>
Committed: Thu Sep 21 21:19:25 2017 -0500

----------------------------------------------------------------------
 .../hbase/chaos/util/ChaosMonkeyRunner.java     | 19 +++-
 .../apache/hadoop/hbase/chaos/util/Monkeys.java | 96 ++++++++++++++++++++
 .../hbase/test/IntegrationTestMonkeys.java      | 67 ++++++++++++++
 3 files changed, 178 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5f238b3e/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java
index 5911085..d72111f 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java
@@ -52,7 +52,6 @@ public class ChaosMonkeyRunner extends AbstractHBaseTool {
   protected boolean noClusterCleanUp = false;
   private String tableName = "ChaosMonkeyRunner.tableName";
   private String familyName = "ChaosMonkeyRunner.familyName";
-  private volatile boolean stop = false;
 
   @Override
   public void addOptions() {
@@ -93,14 +92,26 @@ public class ChaosMonkeyRunner extends AbstractHBaseTool {
   protected int doWork() throws Exception {
     setUpCluster();
     getAndStartMonkey();
-    while (!stop) {// loop here until got killed
-      Thread.sleep(10000);
+    while (!monkey.isStopped()) {
+      // loop here until got killed
+      try {
+        // TODO: make sleep time configurable
+        Thread.sleep(5000); // 5 seconds
+      } catch (InterruptedException ite) {
+        // Chaos monkeys got interrupted.
+        // It is ok to stop monkeys and exit.
+        monkey.stop("Interruption occurred.");
+        break;
+      }
     }
+    monkey.waitForStop();
     return 0;
   }
 
   public void stopRunner() {
-    stop = true;
+    if (monkey != null) {
+      monkey.stop("Program Control");
+    }
   }
 
   public void setUpCluster() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f238b3e/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java
new file mode 100644
index 0000000..1ce4356
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java
@@ -0,0 +1,96 @@
+/**
+ * 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.chaos.util;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class can be used to control chaos monkeys life cycle.
+ */
+public class Monkeys implements Closeable {
+  private static final Log LOG = LogFactory.getLog(Monkeys.class);
+
+  private final Configuration conf;
+  private final ChaosMonkeyRunner monkeyRunner;
+  private final Runnable runner;
+  private final ExecutorService executor;
+
+  public Monkeys() {
+    this(HBaseConfiguration.create());
+  }
+
+  public Monkeys(Configuration conf) {
+    this.conf = Preconditions.checkNotNull(conf, "Should specify a configuration");
+    this.monkeyRunner = new ChaosMonkeyRunner();
+    this.runner = () -> {
+      try {
+        monkeyRunner.getAndStartMonkey();
+      } catch (Exception e) {
+        LOG.error("Exception occured when running chaos monkeys: ", e);
+      }
+    };
+    this.executor = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setDaemon(true).setNameFormat("ChaosMonkey").build());
+    IntegrationTestingUtility.setUseDistributedCluster(this.conf);
+  }
+
+  public void addResource(Configuration otherConf) {
+    conf.addResource(otherConf);
+    monkeyRunner.setConf(conf);
+  }
+
+  public void addResource(String otherConf) {
+    conf.addResource(otherConf);
+    monkeyRunner.setConf(conf);
+  }
+
+  public void startChaos() {
+    executor.execute(runner);
+    LOG.info("Chaos monkeys are running.");
+  }
+
+  public void stopChaos() {
+    monkeyRunner.stopRunner();
+    LOG.info("Chaos monkeys are stopped.");
+  }
+
+  @Override
+  public void close() throws IOException {
+    executor.shutdown();
+    try {
+      // wait 10 seconds.
+      executor.awaitTermination(10, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.warn("Interruption occured while stopping chaos monkeys " + e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f238b3e/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java
new file mode 100644
index 0000000..7b217f6
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.chaos.util.ChaosMonkeyRunner;
+import org.apache.hadoop.hbase.chaos.util.Monkeys;
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * This is an integration test for showing a simple usage of how to use {@link Monkeys}
+ * to control {@link ChaosMonkeyRunner}.
+ */
+@Category(IntegrationTests.class)
+public class IntegrationTestMonkeys extends ChaosMonkeyRunner {
+  private static final int RUN_SECS = 15 * 1000;
+  private static final int WAIT_SECS = 10 * 1000;
+
+  @Override
+  protected int doWork() throws Exception {
+    super.setUpCluster();
+    runMonkeys();
+    return 0;
+  }
+
+  @Test
+  public void runMonkeys() throws Exception {
+    try (Monkeys monkeys = new Monkeys()) {
+      for (int i = 0; i < 2; i++) {
+        monkeys.startChaos();
+        Thread.sleep(RUN_SECS);
+        monkeys.stopChaos();
+        Thread.sleep(WAIT_SECS);
+      }
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    // Run chaos monkeys 15 seconds, then stop them.
+    // After 10 seconds, run chaos monkeys again.
+    Configuration conf = HBaseConfiguration.create();
+    IntegrationTestingUtility.setUseDistributedCluster(conf);
+    int exitCode = ToolRunner.run(conf, new IntegrationTestMonkeys(), args);
+    System.exit(exitCode);
+  }
+}


[15/47] hbase git commit: HBASE-18821 addendum use versionRange instead of version to fix m2e error

Posted by bu...@apache.org.
HBASE-18821 addendum use versionRange instead of version to fix m2e error


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

Branch: refs/heads/HBASE-18467
Commit: f7a986cb67b55e36b58bf4b4934a2f32f29f538a
Parents: 5c07dba
Author: zhangduo <zh...@apache.org>
Authored: Fri Sep 15 16:53:43 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Sep 15 16:54:16 2017 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/f7a986cb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 019501c..af98b51 100755
--- a/pom.xml
+++ b/pom.xml
@@ -812,7 +812,7 @@
                   <pluginExecutionFilter>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-enforcer-plugin</artifactId>
-                    <version>${enforcer.version}</version>
+                    <versionRange>${enforcer.version}</versionRange>
                     <goals>
                       <goal>enforce</goal>
                     </goals>


[14/47] hbase git commit: HBASE-18772 (Addendum) we pass the wrong arguments to AdaptiveLifoCoDelCallQueue

Posted by bu...@apache.org.
HBASE-18772 (Addendum) we pass the wrong arguments to AdaptiveLifoCoDelCallQueue

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


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

Branch: refs/heads/HBASE-18467
Commit: 5c07dba43a9aacd77037948528b45fce4a42d822
Parents: fb3c8bf
Author: Yechao Chen <ch...@gmail.com>
Authored: Thu Sep 14 16:53:35 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Sep 15 10:27:47 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/ipc/RpcExecutor.java     | 10 +++++-----
 .../apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java   |  2 --
 2 files changed, 5 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5c07dba4/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
index 3971fb3..5c017305 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
@@ -26,7 +26,7 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -76,8 +76,8 @@ public abstract class RpcExecutor {
   public static final int CALL_QUEUE_CODEL_DEFAULT_INTERVAL = 100;
   public static final double CALL_QUEUE_CODEL_DEFAULT_LIFO_THRESHOLD = 0.8;
 
-  private AtomicLong numGeneralCallsDropped = new AtomicLong();
-  private AtomicLong numLifoModeSwitches = new AtomicLong();
+  private LongAdder numGeneralCallsDropped = new LongAdder();
+  private LongAdder numLifoModeSwitches = new LongAdder();
 
   protected final int numCallQueues;
   protected final List<BlockingQueue<CallRunner>> queues;
@@ -385,11 +385,11 @@ public abstract class RpcExecutor {
   }
 
   public long getNumGeneralCallsDropped() {
-    return numGeneralCallsDropped.get();
+    return numGeneralCallsDropped.longValue();
   }
 
   public long getNumLifoModeSwitches() {
-    return numLifoModeSwitches.get();
+    return numLifoModeSwitches.longValue();
   }
 
   public int getActiveHandlerCount() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5c07dba4/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 90d5772..b4f93c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -431,7 +430,6 @@ public class TestSimpleRpcScheduler {
   // fastpath thread to: new FastPathBalancedQueueRpcExecutor("CodelFPBQ.default", handlerCount, numCallQueues...
   // Codel is hard to test. This test is going to be flakey given it all timer-based. Disabling for now till chat
   // with authors.
-  @Ignore
   @Test
   public void testCoDelScheduling() throws Exception {
     CoDelEnvironmentEdge envEdge = new CoDelEnvironmentEdge();


[46/47] hbase git commit: HBASE-18866 clean up warnings about proto syntax

Posted by bu...@apache.org.
HBASE-18866 clean up warnings about proto syntax

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


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

Branch: refs/heads/HBASE-18467
Commit: b4830466dbd606ddaf7cc6c87f8eae7141905827
Parents: b27f9b5
Author: Sean Busbey <bu...@apache.org>
Authored: Fri Sep 22 15:35:47 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Sep 22 18:36:08 2017 -0500

----------------------------------------------------------------------
 hbase-protocol-shaded/src/main/protobuf/AccessControl.proto      | 1 +
 hbase-protocol-shaded/src/main/protobuf/Admin.proto              | 2 +-
 hbase-protocol-shaded/src/main/protobuf/Backup.proto             | 4 ++--
 hbase-protocol-shaded/src/main/protobuf/Cell.proto               | 2 +-
 hbase-protocol-shaded/src/main/protobuf/Client.proto             | 2 +-
 hbase-protocol-shaded/src/main/protobuf/ClusterId.proto          | 2 +-
 hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto      | 2 +-
 hbase-protocol-shaded/src/main/protobuf/Comparator.proto         | 1 +
 hbase-protocol-shaded/src/main/protobuf/Encryption.proto         | 1 +
 hbase-protocol-shaded/src/main/protobuf/ErrorHandling.proto      | 1 +
 hbase-protocol-shaded/src/main/protobuf/FS.proto                 | 1 +
 hbase-protocol-shaded/src/main/protobuf/Filter.proto             | 1 +
 hbase-protocol-shaded/src/main/protobuf/HBase.proto              | 1 +
 hbase-protocol-shaded/src/main/protobuf/HFile.proto              | 1 +
 hbase-protocol-shaded/src/main/protobuf/LoadBalancer.proto       | 1 +
 hbase-protocol-shaded/src/main/protobuf/LockService.proto        | 1 +
 hbase-protocol-shaded/src/main/protobuf/MapReduce.proto          | 1 +
 hbase-protocol-shaded/src/main/protobuf/Master.proto             | 1 +
 hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto    | 1 +
 hbase-protocol-shaded/src/main/protobuf/Procedure.proto          | 1 +
 hbase-protocol-shaded/src/main/protobuf/Quota.proto              | 1 +
 hbase-protocol-shaded/src/main/protobuf/RPC.proto                | 1 +
 hbase-protocol-shaded/src/main/protobuf/RegionNormalizer.proto   | 1 +
 hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto | 1 +
 hbase-protocol-shaded/src/main/protobuf/Replication.proto        | 1 +
 hbase-protocol-shaded/src/main/protobuf/Snapshot.proto           | 1 +
 hbase-protocol-shaded/src/main/protobuf/TestProcedure.proto      | 1 +
 hbase-protocol-shaded/src/main/protobuf/Tracing.proto            | 1 +
 hbase-protocol-shaded/src/main/protobuf/WAL.proto                | 1 +
 hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto          | 1 +
 hbase-protocol-shaded/src/main/protobuf/test.proto               | 1 +
 hbase-protocol-shaded/src/main/protobuf/test_rpc_service.proto   | 1 +
 32 files changed, 33 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto b/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto
index 39143e2..3b0e9bb 100644
--- a/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 package hbase.pb;
 
 option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 62aac9a..dc4e324 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+syntax = "proto2";
 // This file contains protocol buffers that are used for Admin service.
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Backup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
index 241d42a..6084e17 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Backup.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+syntax = "proto2";
 // This file contains Backup manifest
 package hbase.pb;
 
@@ -114,4 +114,4 @@ message BackupInfo {
     INCREMENTAL_COPY = 4;
     STORE_MANIFEST = 5;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Cell.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Cell.proto b/hbase-protocol-shaded/src/main/protobuf/Cell.proto
index 82c960f..0e9eb94 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Cell.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Cell.proto
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+syntax = "proto2";
 // Cell and KeyValue protos
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Client.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Client.proto b/hbase-protocol-shaded/src/main/protobuf/Client.proto
index 14d2b4c..e5c843f 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Client.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Client.proto
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+syntax = "proto2";
 // This file contains protocol buffers that are used for Client service.
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/ClusterId.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ClusterId.proto b/hbase-protocol-shaded/src/main/protobuf/ClusterId.proto
index a69b09e..d452aa5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ClusterId.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ClusterId.proto
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+syntax = "proto2";
 // This file contains protocol buffers that are shared throughout HBase
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto b/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
index 52fcc3e..2fb221b 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+syntax = "proto2";
 // This file contains protocol buffers that are used for ClustStatus
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Comparator.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Comparator.proto b/hbase-protocol-shaded/src/main/protobuf/Comparator.proto
index 5e1e2da..822fd2b 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Comparator.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Comparator.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers that are used for filters
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Encryption.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Encryption.proto b/hbase-protocol-shaded/src/main/protobuf/Encryption.proto
index d9ad575..d0b445c 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Encryption.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Encryption.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers used for encryption
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/ErrorHandling.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ErrorHandling.proto b/hbase-protocol-shaded/src/main/protobuf/ErrorHandling.proto
index b08b3b3..367fd85 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ErrorHandling.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ErrorHandling.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers that are used for error handling
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/FS.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/FS.proto b/hbase-protocol-shaded/src/main/protobuf/FS.proto
index 2ef59e2..f32aa04 100644
--- a/hbase-protocol-shaded/src/main/protobuf/FS.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/FS.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers that are written into the filesystem
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Filter.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Filter.proto b/hbase-protocol-shaded/src/main/protobuf/Filter.proto
index 39a7d51..7434985 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Filter.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Filter.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers that are used for filters
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/HBase.proto b/hbase-protocol-shaded/src/main/protobuf/HBase.proto
index 10742ad..9de897a 100644
--- a/hbase-protocol-shaded/src/main/protobuf/HBase.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/HBase.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers that are shared throughout HBase
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/HFile.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/HFile.proto b/hbase-protocol-shaded/src/main/protobuf/HFile.proto
index c88ef17..33f89a2 100644
--- a/hbase-protocol-shaded/src/main/protobuf/HFile.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/HFile.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/LoadBalancer.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/LoadBalancer.proto b/hbase-protocol-shaded/src/main/protobuf/LoadBalancer.proto
index 857d2b6..2a75ca4 100644
--- a/hbase-protocol-shaded/src/main/protobuf/LoadBalancer.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/LoadBalancer.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers to represent the state of the load balancer.
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/LockService.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
index 567dee7..b8d180c 100644
--- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/MapReduce.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MapReduce.proto b/hbase-protocol-shaded/src/main/protobuf/MapReduce.proto
index 26bade5..2496fa7 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MapReduce.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MapReduce.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
  //This file includes protocol buffers used in MapReduce only.
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 6b16bf8..0a13e22 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // All to do with the Master.  Includes schema management since these
 // changes are run by the Master process.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 70753c6..2cdebb1 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 package hbase.pb;
 
 option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
index c13a37e..2c5f1aa 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Procedure.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 package hbase.pb;
 
 option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Quota.proto b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
index 0d74435..cd4c7df 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Quota.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Quota.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/RPC.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RPC.proto b/hbase-protocol-shaded/src/main/protobuf/RPC.proto
index 9cdf98c..1ccf6e8 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RPC.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RPC.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 package hbase.pb;
 
 import "Tracing.proto"; 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/RegionNormalizer.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionNormalizer.proto b/hbase-protocol-shaded/src/main/protobuf/RegionNormalizer.proto
index ed7a9ec..c5a1d41 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionNormalizer.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionNormalizer.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers to represent the state of the load balancer.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index 7d35df0..1cd4376 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // This file contains protocol buffers that are used for RegionServerStatusProtocol.
 package hbase.pb;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Replication.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
index 4cb4436..7e78144 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 package hbase.pb;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto b/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto
index 595a8cf..479e33e 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Snapshot.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 package hbase.pb;
 
 option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/TestProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/TestProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/TestProcedure.proto
index 1b21e8e..e927e4b 100644
--- a/hbase-protocol-shaded/src/main/protobuf/TestProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/TestProcedure.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 option java_package = "org.apache.hadoop.hbase.shaded.ipc.protobuf.generated";
 option java_outer_classname = "TestProcedureProtos";
 option java_generic_services = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Tracing.proto b/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
index 3ec10a8..64ead84 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 package hbase.pb;
 
 option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/WAL.proto b/hbase-protocol-shaded/src/main/protobuf/WAL.proto
index 81e5650..08d4741 100644
--- a/hbase-protocol-shaded/src/main/protobuf/WAL.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/WAL.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 package hbase.pb;
 
 option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
index fcf9bf5..9d3d173 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 // ZNode data in hbase are serialized protobufs with a four byte
 // 'magic' 'PBUF' prefix.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/test.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/test.proto b/hbase-protocol-shaded/src/main/protobuf/test.proto
index 647fafe..5101644 100644
--- a/hbase-protocol-shaded/src/main/protobuf/test.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/test.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 
 option java_package = "org.apache.hadoop.hbase.shaded.ipc.protobuf.generated";
 option java_outer_classname = "TestProtos";

http://git-wip-us.apache.org/repos/asf/hbase/blob/b4830466/hbase-protocol-shaded/src/main/protobuf/test_rpc_service.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/test_rpc_service.proto b/hbase-protocol-shaded/src/main/protobuf/test_rpc_service.proto
index 51472db..03ac385 100644
--- a/hbase-protocol-shaded/src/main/protobuf/test_rpc_service.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/test_rpc_service.proto
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+syntax = "proto2";
 option java_package = "org.apache.hadoop.hbase.shaded.ipc.protobuf.generated";
 option java_outer_classname = "TestRpcServiceProtos";
 option java_generic_services = true;


[40/47] hbase git commit: HBASE-18731 [compat 1-2] Mark protected methods of QuotaSettings that touch Protobuf internals as IA.Private

Posted by bu...@apache.org.
HBASE-18731 [compat 1-2] Mark protected methods of QuotaSettings that touch Protobuf internals as IA.Private

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


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

Branch: refs/heads/HBASE-18467
Commit: e39359986c4765946cde30da2957324cb7c9705c
Parents: a6c3c64
Author: Sean Busbey <bu...@apache.org>
Authored: Thu Sep 21 09:57:39 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Thu Sep 21 14:00:10 2017 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java    | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e3935998/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
index dbea5fb..cf04f92 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettings.java
@@ -56,6 +56,7 @@ public abstract class QuotaSettings {
    * This is used internally by the Admin client to serialize the quota settings
    * and send them to the master.
    */
+  @InterfaceAudience.Private
   public static SetQuotaRequest buildSetQuotaRequestProto(final QuotaSettings settings) {
     SetQuotaRequest.Builder builder = SetQuotaRequest.newBuilder();
     if (settings.getUserName() != null) {
@@ -76,6 +77,7 @@ public abstract class QuotaSettings {
    * the subclass should implement this method to set the specific SetQuotaRequest
    * properties.
    */
+  @InterfaceAudience.Private
   protected abstract void setupSetQuotaRequest(SetQuotaRequest.Builder builder);
 
   protected String ownerToString() {


[11/47] hbase git commit: HBASE-17980 Any HRegionInfo we give out should be immutable

Posted by bu...@apache.org.
HBASE-17980 Any HRegionInfo we give out should be immutable

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


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

Branch: refs/heads/HBASE-18467
Commit: 58988cb53c6287dc82a7a6242b1fe3f50dbf9dfb
Parents: a4277f3
Author: brandboat <br...@gmail.com>
Authored: Wed Sep 13 21:00:18 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 14 14:25:07 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HRegionInfo.java    | 579 +++++---------
 .../org/apache/hadoop/hbase/client/Admin.java   |  25 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  37 +-
 .../hbase/client/ImmutableHRegionInfo.java      |  49 ++
 .../apache/hadoop/hbase/client/RegionInfo.java  | 751 +++++++++++++++++++
 .../hadoop/hbase/client/RegionInfoBuilder.java  | 624 +++++++++++++++
 .../hadoop/hbase/client/RegionInfoDisplay.java  | 135 ++++
 .../hadoop/hbase/client/RegionReplicaUtil.java  |  32 +
 .../hbase/client/UnmodifyableHRegionInfo.java   |   2 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  64 ++
 .../hbase/zookeeper/MetaTableLocator.java       |   5 +-
 .../hbase/client/TestImmutableHRegionInfo.java  |  61 ++
 .../hbase/client/TestRegionInfoDisplay.java     | 129 ++++
 .../master/AssignmentManagerStatusTmpl.jamon    |   4 +-
 .../hbase/master/assignment/RegionStates.java   |  35 +-
 .../apache/hadoop/hbase/MetaMockingUtil.java    |  13 -
 .../master/assignment/TestRegionStates.java     |  24 +-
 .../regionserver/TestRegionInfoBuilder.java     | 323 ++++++++
 18 files changed, 2445 insertions(+), 447 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index 2e735a0..cc88733 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -28,22 +28,17 @@ import java.util.stream.Collectors;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
-import org.apache.hadoop.hbase.util.ByteArrayHashKey;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.HashKey;
-import org.apache.hadoop.hbase.util.JenkinsHash;
-import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.hbase.client.RegionInfoDisplay;
 import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Information about a region. A region is a range of keys in the whole keyspace of a table, an
@@ -74,10 +69,12 @@ import org.apache.hadoop.util.StringUtils;
  * correspond to multiple HRegionInfo's. These HRI's share the same fields however except the
  * replicaId field. If the replicaId is not set, it defaults to 0, which is compatible with the
  * previous behavior of a range corresponding to 1 region.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+ *             use {@link RegionInfoBuilder} to build {@link RegionInfo}.
  */
+@Deprecated
 @InterfaceAudience.Public
-public class HRegionInfo implements Comparable<HRegionInfo> {
-
+public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
   private static final Log LOG = LogFactory.getLog(HRegionInfo.class);
 
   /**
@@ -103,62 +100,20 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * old region name format.
    */
 
-  /** Separator used to demarcate the encodedName in a region name
-   * in the new format. See description on new format above.
-   */
-  private static final int ENC_SEPARATOR = '.';
-  public  static final int MD5_HEX_LENGTH   = 32;
-
   /** A non-capture group so that this can be embedded. */
-  public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
-
-  // to keep appended int's sorted in string format. Only allows 2 bytes to be
-  // sorted for replicaId
-  public static final String REPLICA_ID_FORMAT = "%04X";
-
-  public static final byte REPLICA_ID_DELIMITER = (byte)'_';
+  public static final String ENCODED_REGION_NAME_REGEX = RegionInfoBuilder.ENCODED_REGION_NAME_REGEX;
 
   private static final int MAX_REPLICA_ID = 0xFFFF;
-  public static final int DEFAULT_REPLICA_ID = 0;
-
-  public static final String INVALID_REGION_NAME_FORMAT_MESSAGE = "Invalid regionName format";
-
-  /**
-   * Does region name contain its encoded name?
-   * @param regionName region name
-   * @return boolean indicating if this a new format region
-   *         name which contains its encoded name.
-   */
-  private static boolean hasEncodedName(final byte[] regionName) {
-    // check if region name ends in ENC_SEPARATOR
-    if ((regionName.length >= 1)
-        && (regionName[regionName.length - 1] == ENC_SEPARATOR)) {
-      // region name is new format. it contains the encoded name.
-      return true;
-    }
-    return false;
-  }
 
   /**
    * @param regionName
    * @return the encodedName
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#encodeRegionName(byte[])}.
    */
+  @Deprecated
   public static String encodeRegionName(final byte [] regionName) {
-    String encodedName;
-    if (hasEncodedName(regionName)) {
-      // region is in new format:
-      // <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
-      encodedName = Bytes.toString(regionName,
-          regionName.length - MD5_HEX_LENGTH - 1,
-          MD5_HEX_LENGTH);
-    } else {
-      // old format region name. First hbase:meta region also
-      // use this format.EncodedName is the JenkinsHash value.
-      HashKey<byte[]> key = new ByteArrayHashKey(regionName, 0, regionName.length);
-      int hashVal = Math.abs(JenkinsHash.getInstance().hash(key, 0));
-      encodedName = String.valueOf(hashVal);
-    }
-    return encodedName;
+    return RegionInfo.encodeRegionName(regionName);
   }
 
   /**
@@ -168,17 +123,24 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     return prettyPrint(this.getEncodedName());
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(RegionInfo...)}.
+   */
+  @Deprecated
   public static String getShortNameToLog(HRegionInfo...hris) {
-    return getShortNameToLog(Arrays.asList(hris));
+    return RegionInfo.getShortNameToLog(Arrays.asList(hris));
   }
 
   /**
    * @return Return a String of short, printable names for <code>hris</code>
    * (usually encoded name) for us logging.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(List)})}.
    */
+  @Deprecated
   public static String getShortNameToLog(final List<HRegionInfo> hris) {
-    return hris.stream().map(hri -> hri.getShortNameToLog()).
-        collect(Collectors.toList()).toString();
+    return RegionInfo.getShortNameToLog(hris.stream().collect(Collectors.toList()));
   }
 
   /**
@@ -186,12 +148,13 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param encodedRegionName The encoded regionname.
    * @return <code>hbase:meta</code> if passed <code>1028785192</code> else returns
    * <code>encodedRegionName</code>
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#prettyPrint(String)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static String prettyPrint(final String encodedRegionName) {
-    if (encodedRegionName.equals("1028785192")) {
-      return encodedRegionName + "/hbase:meta";
-    }
-    return encodedRegionName;
+    return RegionInfo.prettyPrint(encodedRegionName);
   }
 
   private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
@@ -212,9 +175,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
 
   // Current TableName
   private TableName tableName = null;
-  final static String DISPLAY_KEYS_KEY = "hbase.display.keys";
-  public final static byte[] HIDDEN_END_KEY = Bytes.toBytes("hidden-end-key");
-  public final static byte[] HIDDEN_START_KEY = Bytes.toBytes("hidden-start-key");
+
+  // Duplicated over in RegionInfoDisplay
+  final static String DISPLAY_KEYS_KEY = RegionInfoDisplay.DISPLAY_KEYS_KEY;
+  public final static byte[] HIDDEN_END_KEY = RegionInfoDisplay.HIDDEN_END_KEY;
+  public final static byte[] HIDDEN_START_KEY = RegionInfoDisplay.HIDDEN_START_KEY;
 
   /** HRegionInfo for first meta region */
   // TODO: How come Meta regions still do not have encoded region names? Fix.
@@ -232,7 +197,6 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     this.hashCode = result;
   }
 
-
   /**
    * Private constructor used constructing HRegionInfo for the
    * first meta regions
@@ -354,8 +318,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     this.startKey = other.getStartKey();
     this.hashCode = other.hashCode();
     this.encodedName = other.getEncodedName();
-    this.tableName = other.tableName;
-    this.replicaId = other.replicaId;
+    this.tableName = other.getTable();
+    this.replicaId = other.getReplicaId();
   }
 
   public HRegionInfo(HRegionInfo other, int replicaId) {
@@ -372,10 +336,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey and id
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], long, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final long regionid, boolean newFormat) {
-    return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
+    return RegionInfo.createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
   }
 
   /**
@@ -386,10 +354,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey and id
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], String, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final String id, boolean newFormat) {
-    return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
+    return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
   }
 
   /**
@@ -401,10 +373,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey, id and replicaId
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], long, int, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final long regionid, int replicaId, boolean newFormat) {
-    return createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
+    return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
         replicaId, newFormat);
   }
 
@@ -416,10 +392,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey and id
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final byte [] id, boolean newFormat) {
-    return createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
+    return RegionInfo.createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
   }
   /**
    * Make a region name of passed parameters.
@@ -429,94 +409,38 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param replicaId
    * @param newFormat should we create the region name in the new format
    * @return Region name made of passed tableName, startKey, id and replicaId
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], int, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final byte [] id, final int replicaId, boolean newFormat) {
-    int len = tableName.getName().length + 2 + id.length +
-        (startKey == null? 0: startKey.length);
-    if (newFormat) {
-      len += MD5_HEX_LENGTH + 2;
-    }
-    byte[] replicaIdBytes = null;
-    // Special casing: replicaId is only appended if replicaId is greater than
-    // 0. This is because all regions in meta would have to be migrated to the new
-    // name otherwise
-    if (replicaId > 0) {
-      // use string representation for replica id
-      replicaIdBytes = Bytes.toBytes(String.format(REPLICA_ID_FORMAT, replicaId));
-      len += 1 + replicaIdBytes.length;
-    }
-
-    byte [] b = new byte [len];
-
-    int offset = tableName.getName().length;
-    System.arraycopy(tableName.getName(), 0, b, 0, offset);
-    b[offset++] = HConstants.DELIMITER;
-    if (startKey != null && startKey.length > 0) {
-      System.arraycopy(startKey, 0, b, offset, startKey.length);
-      offset += startKey.length;
-    }
-    b[offset++] = HConstants.DELIMITER;
-    System.arraycopy(id, 0, b, offset, id.length);
-    offset += id.length;
-
-    if (replicaIdBytes != null) {
-      b[offset++] = REPLICA_ID_DELIMITER;
-      System.arraycopy(replicaIdBytes, 0, b, offset, replicaIdBytes.length);
-      offset += replicaIdBytes.length;
-    }
-
-    if (newFormat) {
-      //
-      // Encoded name should be built into the region name.
-      //
-      // Use the region name thus far (namely, <tablename>,<startKey>,<id>_<replicaId>)
-      // to compute a MD5 hash to be used as the encoded name, and append
-      // it to the byte buffer.
-      //
-      String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
-      byte [] md5HashBytes = Bytes.toBytes(md5Hash);
-
-      if (md5HashBytes.length != MD5_HEX_LENGTH) {
-        LOG.error("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
-                  "; Got=" + md5HashBytes.length);
-      }
-
-      // now append the bytes '.<encodedName>.' to the end
-      b[offset++] = ENC_SEPARATOR;
-      System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
-      offset += MD5_HEX_LENGTH;
-      b[offset++] = ENC_SEPARATOR;
-    }
-
-    return b;
+    return RegionInfo.createRegionName(tableName, startKey, id, replicaId, newFormat);
   }
 
   /**
    * Gets the table name from the specified region name.
    * @param regionName to extract the table name from
    * @return Table name
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getTable(byte[])}.
    */
+  @Deprecated
   public static TableName getTable(final byte [] regionName) {
-    int offset = -1;
-    for (int i = 0; i < regionName.length; i++) {
-      if (regionName[i] == HConstants.DELIMITER) {
-        offset = i;
-        break;
-      }
-    }
-    byte[] buff  = new byte[offset];
-    System.arraycopy(regionName, 0, buff, 0, offset);
-    return TableName.valueOf(buff);
+    return RegionInfo.getTable(regionName);
   }
 
   /**
    * Gets the start key from the specified region name.
    * @param regionName
    * @return Start key.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getStartKey(byte[])}.
    */
+  @Deprecated
   public static byte[] getStartKey(final byte[] regionName) throws IOException {
-    return parseRegionName(regionName)[1];
+    return RegionInfo.getStartKey(regionName);
   }
 
   /**
@@ -524,88 +448,27 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param regionName
    * @return Array of byte[] containing tableName, startKey and id
    * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#parseRegionName(byte[])}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [][] parseRegionName(final byte [] regionName)
   throws IOException {
-    // Region name is of the format:
-    // tablename,startkey,regionIdTimestamp[_replicaId][.encodedName.]
-    // startkey can contain the delimiter (',') so we parse from the start and end
-
-    // parse from start
-    int offset = -1;
-    for (int i = 0; i < regionName.length; i++) {
-      if (regionName[i] == HConstants.DELIMITER) {
-        offset = i;
-        break;
-      }
-    }
-    if (offset == -1) {
-      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
-        + ": " + Bytes.toStringBinary(regionName));
-    }
-    byte[] tableName = new byte[offset];
-    System.arraycopy(regionName, 0, tableName, 0, offset);
-    offset = -1;
-
-    int endOffset = regionName.length;
-    // check whether regionName contains encodedName
-    if (regionName.length > MD5_HEX_LENGTH + 2
-        && regionName[regionName.length-1] == ENC_SEPARATOR
-        && regionName[regionName.length-MD5_HEX_LENGTH-2] == ENC_SEPARATOR) {
-      endOffset = endOffset - MD5_HEX_LENGTH - 2;
-    }
-
-    // parse from end
-    byte[] replicaId = null;
-    int idEndOffset = endOffset;
-    for (int i = endOffset - 1; i > 0; i--) {
-      if (regionName[i] == REPLICA_ID_DELIMITER) { //replicaId may or may not be present
-        replicaId = new byte[endOffset - i - 1];
-        System.arraycopy(regionName, i + 1, replicaId, 0,
-          endOffset - i - 1);
-        idEndOffset = i;
-        // do not break, continue to search for id
-      }
-      if (regionName[i] == HConstants.DELIMITER) {
-        offset = i;
-        break;
-      }
-    }
-    if (offset == -1) {
-      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
-        + ": " + Bytes.toStringBinary(regionName));
-    }
-    byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
-    if(offset != tableName.length + 1) {
-      startKey = new byte[offset - tableName.length - 1];
-      System.arraycopy(regionName, tableName.length + 1, startKey, 0,
-          offset - tableName.length - 1);
-    }
-    byte [] id = new byte[idEndOffset - offset - 1];
-    System.arraycopy(regionName, offset + 1, id, 0,
-      idEndOffset - offset - 1);
-    byte [][] elements = new byte[replicaId == null ? 3 : 4][];
-    elements[0] = tableName;
-    elements[1] = startKey;
-    elements[2] = id;
-    if (replicaId != null) {
-      elements[3] = replicaId;
-    }
-
-    return elements;
+    return RegionInfo.parseRegionName(regionName);
   }
 
+  /**
+   *
+   * @param regionName
+   * @return if region name is encoded.
+   * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#isEncodedRegionName(byte[])}.
+   */
+  @Deprecated
   public static boolean isEncodedRegionName(byte[] regionName) throws IOException {
-    try {
-      HRegionInfo.parseRegionName(regionName);
-      return false;
-    } catch (IOException e) {
-      if (StringUtils.stringifyException(e)
-          .contains(HRegionInfo.INVALID_REGION_NAME_FORMAT_MESSAGE)) {
-        return true;
-      }
-      throw e;
-    }
+    return RegionInfo.isEncodedRegionName(regionName);
   }
 
   /** @return the regionId */
@@ -625,7 +488,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return Region name as a String for use in logging, etc.
    */
   public String getRegionNameAsString() {
-    if (hasEncodedName(this.regionName)) {
+    if (RegionInfo.hasEncodedName(this.regionName)) {
       // new format region names already have their encoded name.
       return Bytes.toStringBinary(this.regionName);
     }
@@ -639,7 +502,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
   /** @return the encoded region name */
   public synchronized String getEncodedName() {
     if (this.encodedName == null) {
-      this.encodedName = encodeRegionName(this.regionName);
+      this.encodedName = RegionInfo.encodeRegionName(this.regionName);
     }
     return this.encodedName;
   }
@@ -819,53 +682,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
 
   @Override
   public int compareTo(HRegionInfo o) {
-    if (o == null) {
-      return 1;
-    }
-
-    // Are regions of same table?
-    int result = this.tableName.compareTo(o.tableName);
-    if (result != 0) {
-      return result;
-    }
-
-    // Compare start keys.
-    result = Bytes.compareTo(this.startKey, o.startKey);
-    if (result != 0) {
-      return result;
-    }
-
-    // Compare end keys.
-    result = Bytes.compareTo(this.endKey, o.endKey);
-
-    if (result != 0) {
-      if (this.getStartKey().length != 0
-              && this.getEndKey().length == 0) {
-          return 1; // this is last region
-      }
-      if (o.getStartKey().length != 0
-              && o.getEndKey().length == 0) {
-          return -1; // o is the last region
-      }
-      return result;
-    }
-
-    // regionId is usually milli timestamp -- this defines older stamps
-    // to be "smaller" than newer stamps in sort order.
-    if (this.regionId > o.regionId) {
-      return 1;
-    } else if (this.regionId < o.regionId) {
-      return -1;
-    }
-
-    int replicaDiff = this.getReplicaId() - o.getReplicaId();
-    if (replicaDiff != 0) return replicaDiff;
-
-    if (this.offLine == o.offLine)
-      return 0;
-    if (this.offLine == true) return -1;
-
-    return 1;
+    return RegionInfo.COMPARATOR.compare(this, o);
   }
 
   /**
@@ -883,7 +700,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    *
    * @return the converted RegionInfo
    */
-  RegionInfo convert() {
+  HBaseProtos.RegionInfo convert() {
     return convert(this);
   }
 
@@ -892,58 +709,47 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    *
    * @param info the HRegionInfo to convert
    * @return the converted RegionInfo
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use toProtoRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
+   *             in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
    */
-  public static RegionInfo convert(final HRegionInfo info) {
-    if (info == null) return null;
-    RegionInfo.Builder builder = RegionInfo.newBuilder();
-    builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
-    builder.setRegionId(info.getRegionId());
-    if (info.getStartKey() != null) {
-      builder.setStartKey(UnsafeByteOperations.unsafeWrap(info.getStartKey()));
-    }
-    if (info.getEndKey() != null) {
-      builder.setEndKey(UnsafeByteOperations.unsafeWrap(info.getEndKey()));
-    }
-    builder.setOffline(info.isOffline());
-    builder.setSplit(info.isSplit());
-    builder.setReplicaId(info.getReplicaId());
-    return builder.build();
+  @Deprecated
+  @InterfaceAudience.Private
+  public static HBaseProtos.RegionInfo convert(final HRegionInfo info) {
+    return ProtobufUtil.toProtoRegionInfo(info);
   }
 
   /**
    * Convert a RegionInfo to a HRegionInfo
    *
    * @param proto the RegionInfo to convert
-   * @return the converted HRegionInfho
+   * @return the converted HRegionInfo
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use toRegionInfo(HBaseProtos.RegionInfo)
+   *             in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
    */
-  public static HRegionInfo convert(final RegionInfo proto) {
-    if (proto == null) return null;
-    TableName tableName =
-        ProtobufUtil.toTableName(proto.getTableName());
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      return RegionReplicaUtil.getRegionInfoForReplica(FIRST_META_REGIONINFO,
-          proto.getReplicaId());
-    }
-    long regionId = proto.getRegionId();
-    int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : DEFAULT_REPLICA_ID;
-    byte[] startKey = null;
-    byte[] endKey = null;
-    if (proto.hasStartKey()) {
-      startKey = proto.getStartKey().toByteArray();
-    }
-    if (proto.hasEndKey()) {
-      endKey = proto.getEndKey().toByteArray();
-    }
-    boolean split = false;
-    if (proto.hasSplit()) {
-      split = proto.getSplit();
-    }
-    HRegionInfo hri = new HRegionInfo(
-        tableName,
-        startKey,
-        endKey, split, regionId, replicaId);
-    if (proto.hasOffline()) {
-      hri.setOffline(proto.getOffline());
+  @Deprecated
+  @InterfaceAudience.Private
+  public static HRegionInfo convert(final HBaseProtos.RegionInfo proto) {
+    RegionInfo ri = ProtobufUtil.toRegionInfo(proto);
+    // This is hack of what is in RegionReplicaUtil but it is doing translation of
+    // RegionInfo into HRegionInfo which is what is wanted here.
+    HRegionInfo hri;
+    if (ri.isMetaRegion()) {
+      hri = ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID ?
+      HRegionInfo.FIRST_META_REGIONINFO :
+      new HRegionInfo(ri.getRegionId(), ri.getTable(), ri.getReplicaId());
+    } else {
+      hri = new HRegionInfo(
+        ri.getTable(),
+        ri.getStartKey(),
+        ri.getEndKey(),
+        ri.isSplit(),
+        ri.getRegionId(),
+        ri.getReplicaId());
+      if (proto.hasOffline()) {
+        hri.setOffline(proto.getOffline());
+      }
     }
     return hri;
   }
@@ -951,17 +757,22 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
   /**
    * @return This instance serialized as protobuf w/ a magic pb prefix.
    * @see #parseFrom(byte[])
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#toByteArray(RegionInfo)}.
    */
+  @Deprecated
   public byte [] toByteArray() {
-    byte [] bytes = convert().toByteArray();
-    return ProtobufUtil.prependPBMagic(bytes);
+    return RegionInfo.toByteArray(this);
   }
 
   /**
    * @return A deserialized {@link HRegionInfo}
    * or null if we failed deserialize or passed bytes null
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[])}.
    */
+  @Deprecated
   public static HRegionInfo parseFromOrNull(final byte [] bytes) {
     if (bytes == null) return null;
     return parseFromOrNull(bytes, 0, bytes.length);
@@ -971,7 +782,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return A deserialized {@link HRegionInfo} or null
    *  if we failed deserialize or passed bytes null
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[], int, int)}.
    */
+  @Deprecated
   public static HRegionInfo parseFromOrNull(final byte [] bytes, int offset, int len) {
     if (bytes == null || len <= 0) return null;
     try {
@@ -986,6 +800,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return A deserialized {@link HRegionInfo}
    * @throws DeserializationException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[])}.
    */
   public static HRegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
     if (bytes == null) return null;
@@ -999,7 +815,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return A deserialized {@link HRegionInfo}
    * @throws DeserializationException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[], int, int)}.
    */
+  @Deprecated
   public static HRegionInfo parseFrom(final byte [] bytes, int offset, int len)
       throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) {
@@ -1023,9 +842,12 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
    * @throws IOException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#toDelimitedByteArray(RegionInfo)}.
    */
+  @Deprecated
   public byte [] toDelimitedByteArray() throws IOException {
-    return ProtobufUtil.toDelimitedByteArray(convert());
+    return RegionInfo.toDelimitedByteArray(this);
   }
 
   /**
@@ -1034,14 +856,15 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param state
    * @param conf
    * @return descriptive string
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getDescriptiveNameFromRegionStateForDisplay(RegionState, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static String getDescriptiveNameFromRegionStateForDisplay(RegionState state,
       Configuration conf) {
-    if (conf.getBoolean(DISPLAY_KEYS_KEY, true)) return state.toDescriptiveString();
-    String descriptiveStringFromState = state.toDescriptiveString();
-    int idx = descriptiveStringFromState.lastIndexOf(" state=");
-    String regionName = getRegionNameAsStringForDisplay(state.getRegion(), conf);
-    return regionName + descriptiveStringFromState.substring(idx);
+    return RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
   }
 
   /**
@@ -1049,11 +872,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return the endkey
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getEndKeyForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] getEndKeyForDisplay(HRegionInfo hri, Configuration conf) {
-    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
-    if (displayKey) return hri.getEndKey();
-    return HIDDEN_END_KEY;
+    return RegionInfoDisplay.getEndKeyForDisplay(hri, conf);
   }
 
   /**
@@ -1061,11 +887,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return the startkey
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getStartKeyForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] getStartKeyForDisplay(HRegionInfo hri, Configuration conf) {
-    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
-    if (displayKey) return hri.getStartKey();
-    return HIDDEN_START_KEY;
+    return RegionInfoDisplay.getStartKeyForDisplay(hri, conf);
   }
 
   /**
@@ -1073,9 +902,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return region name as String
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getRegionNameAsStringForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static String getRegionNameAsStringForDisplay(HRegionInfo hri, Configuration conf) {
-    return Bytes.toStringBinary(getRegionNameForDisplay(hri, conf));
+    return RegionInfoDisplay.getRegionNameAsStringForDisplay(hri, conf);
   }
 
   /**
@@ -1083,47 +917,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return region name bytes
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getRegionNameForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] getRegionNameForDisplay(HRegionInfo hri, Configuration conf) {
-    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
-    if (displayKey || hri.getTable().equals(TableName.META_TABLE_NAME)) {
-      return hri.getRegionName();
-    } else {
-      // create a modified regionname with the startkey replaced but preserving
-      // the other parts including the encodedname.
-      try {
-        byte[][]regionNameParts = parseRegionName(hri.getRegionName());
-        regionNameParts[1] = HIDDEN_START_KEY; //replace the real startkey
-        int len = 0;
-        // get the total length
-        for (byte[] b : regionNameParts) {
-          len += b.length;
-        }
-        byte[] encodedRegionName =
-            Bytes.toBytes(encodeRegionName(hri.getRegionName()));
-        len += encodedRegionName.length;
-        //allocate some extra bytes for the delimiters and the last '.'
-        byte[] modifiedName = new byte[len + regionNameParts.length + 1];
-        int lengthSoFar = 0;
-        int loopCount = 0;
-        for (byte[] b : regionNameParts) {
-          System.arraycopy(b, 0, modifiedName, lengthSoFar, b.length);
-          lengthSoFar += b.length;
-          if (loopCount++ == 2) modifiedName[lengthSoFar++] = REPLICA_ID_DELIMITER;
-          else  modifiedName[lengthSoFar++] = HConstants.DELIMITER;
-        }
-        // replace the last comma with '.'
-        modifiedName[lengthSoFar - 1] = ENC_SEPARATOR;
-        System.arraycopy(encodedRegionName, 0, modifiedName, lengthSoFar,
-            encodedRegionName.length);
-        lengthSoFar += encodedRegionName.length;
-        modifiedName[lengthSoFar] = ENC_SEPARATOR;
-        return modifiedName;
-      } catch (IOException e) {
-        //LOG.warn("Encountered exception " + e);
-        throw new RuntimeException(e);
-      }
-    }
+    return RegionInfoDisplay.getRegionNameForDisplay(hri, conf);
   }
 
   /**
@@ -1132,7 +933,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param in
    * @return An instance of HRegionInfo.
    * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#parseFrom(DataInputStream)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static HRegionInfo parseFrom(final DataInputStream in) throws IOException {
     // I need to be able to move back in the stream if this is not a pb serialization so I can
     // do the Writable decoding instead.
@@ -1161,22 +966,13 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
    * @throws IOException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#toDelimitedByteArray(RegionInfo...)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] toDelimitedByteArray(HRegionInfo... infos) throws IOException {
-    byte[][] bytes = new byte[infos.length][];
-    int size = 0;
-    for (int i = 0; i < infos.length; i++) {
-      bytes[i] = infos[i].toDelimitedByteArray();
-      size += bytes[i].length;
-    }
-
-    byte[] result = new byte[size];
-    int offset = 0;
-    for (byte[] b : bytes) {
-      System.arraycopy(b, 0, result, offset, b.length);
-      offset += b.length;
-    }
-    return result;
+    return RegionInfo.toDelimitedByteArray(infos);
   }
 
   /**
@@ -1186,7 +982,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param offset the start offset into the byte[] buffer
    * @param length how far we should read into the byte[] buffer
    * @return All the hregioninfos that are in the byte array. Keeps reading till we hit the end.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#parseDelimitedFrom(byte[], int, int)}.
    */
+  @Deprecated
   public static List<HRegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
       final int length) throws IOException {
     if (bytes == null) {
@@ -1211,21 +1010,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param regionA
    * @param regionB
    * @return true if two regions are adjacent
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#areAdjacent(RegionInfo, RegionInfo)}.
    */
+  @Deprecated
   public static boolean areAdjacent(HRegionInfo regionA, HRegionInfo regionB) {
-    if (regionA == null || regionB == null) {
-      throw new IllegalArgumentException(
-          "Can't check whether adjacent for null region");
-    }
-    HRegionInfo a = regionA;
-    HRegionInfo b = regionB;
-    if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) {
-      a = regionB;
-      b = regionA;
-    }
-    if (Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0) {
-      return true;
-    }
-    return false;
+    return RegionInfo.areAdjacent(regionA, regionB);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 0a82a6b..2a92409 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -811,10 +811,22 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Get all the online regions on a region server.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
+   *             Use {@link #getRegions(ServerName sn)}.
    */
+  @Deprecated
   List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException;
 
   /**
+   * Get all the online regions on a region server.
+   *
+   * @return List of {@link RegionInfo}
+   * @throws java.io.IOException
+   */
+  List<RegionInfo> getRegions(ServerName serverName) throws IOException;
+
+  /**
    * Flush a table. Synchronous operation.
    *
    * @param tableName table to flush
@@ -1510,10 +1522,23 @@ public interface Admin extends Abortable, Closeable {
    * @param tableName the name of the table
    * @return List of {@link HRegionInfo}.
    * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
+   *             Use {@link #getRegions(TableName)}.
    */
+  @Deprecated
   List<HRegionInfo> getTableRegions(TableName tableName)
     throws IOException;
 
+  /**
+   * Get the regions of a given table.
+   *
+   * @param tableName the name of the table
+   * @return List of {@link RegionInfo}.
+   * @throws IOException
+   */
+  List<RegionInfo> getRegions(TableName tableName) throws IOException;
+
   @Override
   void close() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 4e17335..fac3ef1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -397,6 +397,16 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  @Override
+  public List<RegionInfo> getRegions(final ServerName sn) throws IOException {
+    return getOnlineRegions(sn).stream().collect(Collectors.toList());
+  }
+
+  @Override
+  public List<RegionInfo> getRegions(final TableName tableName) throws IOException {
+    return getTableRegions(tableName).stream().collect(Collectors.toList());
+  }
+
   private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
     private boolean isAbortInProgress;
 
@@ -1143,12 +1153,24 @@ public class HBaseAdmin implements Admin {
     unassign(hri.getRegionName(), true);
   }
 
+  /**
+   *
+   * @param sn
+   * @return List of {@link HRegionInfo}.
+   * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegions(ServerName)}.
+   */
+  @Deprecated
   @Override
   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // TODO: There is no timeout on this controller. Set one!
     HBaseRpcController controller = rpcControllerFactory.newController();
-    return ProtobufUtil.getOnlineRegions(controller, admin);
+    List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(controller, admin);
+    return onlineRegions == null ? null : onlineRegions.stream()
+            .map(hri -> new ImmutableHRegionInfo(hri))
+            .collect(Collectors.toList());
   }
 
   @Override
@@ -2340,6 +2362,15 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   *
+   * @param tableName
+   * @return List of {@link HRegionInfo}.
+   * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegions(TableName)}.
+   */
+  @Deprecated
   @Override
   public List<HRegionInfo> getTableRegions(final TableName tableName)
   throws IOException {
@@ -2356,7 +2387,9 @@ public class HBaseAdmin implements Admin {
     } finally {
       zookeeper.close();
     }
-    return regions;
+    return regions == null ? null : regions.stream()
+            .map(hri -> new ImmutableHRegionInfo(hri))
+            .collect(Collectors.toList());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
new file mode 100644
index 0000000..16329c8
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
@@ -0,0 +1,49 @@
+/**
+ *
+ * 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.HRegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Read-only Region info.
+ */
+@Deprecated // deprecated for hbase 2.0, remove for hbase 3.0. see HRegionInfo.
+@InterfaceAudience.Private
+public class ImmutableHRegionInfo extends HRegionInfo {
+
+  /*
+   * Creates an immutable copy of an HRegionInfo.
+   *
+   * @param other
+   */
+  public ImmutableHRegionInfo(HRegionInfo other) {
+    super(other);
+  }
+
+  @Override
+  public void setSplit(boolean split) {
+    throw new UnsupportedOperationException("HRegionInfo is read-only");
+  }
+
+  @Override
+  public void setOffline(boolean offline) {
+    throw new UnsupportedOperationException("HRegionInfo is read-only");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
new file mode 100644
index 0000000..3646722
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -0,0 +1,751 @@
+/**
+ *
+ * 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.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.util.ByteArrayHashKey;
+import org.apache.hadoop.hbase.util.HashKey;
+import org.apache.hadoop.hbase.util.JenkinsHash;
+
+/**
+ * Information about a region. A region is a range of keys in the whole keyspace
+ * of a table, an identifier (a timestamp) for differentiating between subset
+ * ranges (after region split) and a replicaId for differentiating the instance
+ * for the same range and some status information about the region.
+ *
+ * The region has a unique name which consists of the following fields:
+ * <ul>
+ * <li> tableName   : The name of the table </li>
+ * <li> startKey    : The startKey for the region. </li>
+ * <li> regionId    : A timestamp when the region is created. </li>
+ * <li> replicaId   : An id starting from 0 to differentiate replicas of the
+ * same region range but hosted in separated servers. The same region range can
+ * be hosted in multiple locations.</li>
+ * <li> encodedName : An MD5 encoded string for the region name.</li>
+ * </ul>
+ *
+ * <br> Other than the fields in the region name, region info contains:
+ * <ul>
+ * <li> endKey      : the endKey for the region (exclusive) </li>
+ * <li> split       : Whether the region is split </li>
+ * <li> offline     : Whether the region is offline </li>
+ * </ul>
+ *
+ */
+@InterfaceAudience.Public
+public interface RegionInfo {
+  /**
+   * Separator used to demarcate the encodedName in a region name
+   * in the new format. See description on new format above.
+   */
+  @InterfaceAudience.Private
+  int ENC_SEPARATOR = '.';
+
+  @InterfaceAudience.Private
+  int MD5_HEX_LENGTH = 32;
+
+  @InterfaceAudience.Private
+  int DEFAULT_REPLICA_ID = 0;
+
+  /**
+   * to keep appended int's sorted in string format. Only allows 2 bytes
+   * to be sorted for replicaId.
+   */
+  @InterfaceAudience.Private
+  String REPLICA_ID_FORMAT = "%04X";
+
+  @InterfaceAudience.Private
+  byte REPLICA_ID_DELIMITER = (byte)'_';
+
+  @InterfaceAudience.Private
+  String INVALID_REGION_NAME_FORMAT_MESSAGE = "Invalid regionName format";
+
+  @InterfaceAudience.Private
+  Comparator<RegionInfo> COMPARATOR
+    = (RegionInfo lhs, RegionInfo rhs) -> {
+      if (rhs == null) {
+        return 1;
+      }
+
+      // Are regions of same table?
+      int result = lhs.getTable().compareTo(rhs.getTable());
+      if (result != 0) {
+        return result;
+      }
+
+      // Compare start keys.
+      result = Bytes.compareTo(lhs.getStartKey(), rhs.getStartKey());
+      if (result != 0) {
+        return result;
+      }
+
+      // Compare end keys.
+      result = Bytes.compareTo(lhs.getEndKey(), rhs.getEndKey());
+
+      if (result != 0) {
+        if (lhs.getStartKey().length != 0
+                && lhs.getEndKey().length == 0) {
+            return 1; // this is last region
+        }
+        if (rhs.getStartKey().length != 0
+                && rhs.getEndKey().length == 0) {
+            return -1; // o is the last region
+        }
+        return result;
+      }
+
+      // regionId is usually milli timestamp -- this defines older stamps
+      // to be "smaller" than newer stamps in sort order.
+      if (lhs.getRegionId() > rhs.getRegionId()) {
+        return 1;
+      } else if (lhs.getRegionId() < rhs.getRegionId()) {
+        return -1;
+      }
+
+      int replicaDiff = lhs.getReplicaId() - rhs.getReplicaId();
+      if (replicaDiff != 0) return replicaDiff;
+
+      if (lhs.isOffline() == rhs.isOffline())
+        return 0;
+      if (lhs.isOffline() == true) return -1;
+
+      return 1;
+  };
+
+
+  /**
+   * @return Return a short, printable name for this region
+   * (usually encoded name) for us logging.
+   */
+  String getShortNameToLog();
+
+  /**
+   * @return the regionId.
+   */
+  long getRegionId();
+
+  /**
+   * @return the regionName as an array of bytes.
+   * @see #getRegionNameAsString()
+   */
+  byte [] getRegionName();
+
+  /**
+   * @return Region name as a String for use in logging, etc.
+   */
+  String getRegionNameAsString();
+
+  /**
+   * @return the encoded region name.
+   */
+  String getEncodedName();
+
+  /**
+   * @return the encoded region name as an array of bytes.
+   */
+  byte [] getEncodedNameAsBytes();
+
+  /**
+   * @return the startKey.
+   */
+  byte [] getStartKey();
+
+  /**
+   * @return the endKey.
+   */
+  byte [] getEndKey();
+
+  /**
+   * @return current table name of the region
+   */
+  TableName getTable();
+
+  /**
+   * @return returns region replica id
+   */
+  int getReplicaId();
+
+  /**
+   * @return True if has been split and has daughters.
+   */
+  boolean isSplit();
+
+  /**
+   * @return True if this region is offline.
+   */
+  boolean isOffline();
+
+  /**
+   * @return True if this is a split parent region.
+   */
+  boolean isSplitParent();
+
+  /**
+   * @return true if this region is from hbase:meta.
+   */
+  boolean isMetaTable();
+
+  /**
+   * @return true if this region is from a system table.
+   */
+  boolean isSystemTable();
+
+  /**
+   * @return true if this region is a meta region.
+   */
+  boolean isMetaRegion();
+
+  /**
+   * @param rangeStartKey
+   * @param rangeEndKey
+   * @return true if the given inclusive range of rows is fully contained
+   * by this region. For example, if the region is foo,a,g and this is
+   * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
+   * ["b","z"] it will return false.
+   * @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
+   */
+  boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey);
+
+  /**
+   * @param row
+   * @return true if the given row falls in this region.
+   */
+  boolean containsRow(byte[] row);
+
+  /**
+   * Does region name contain its encoded name?
+   * @param regionName region name
+   * @return boolean indicating if this a new format region
+   *         name which contains its encoded name.
+   */
+  @InterfaceAudience.Private
+  static boolean hasEncodedName(final byte[] regionName) {
+    // check if region name ends in ENC_SEPARATOR
+    return (regionName.length >= 1) &&
+      (regionName[regionName.length - 1] == RegionInfo.ENC_SEPARATOR);
+  }
+
+  /**
+   * @return the encodedName
+   */
+  @InterfaceAudience.Private
+  static String encodeRegionName(final byte [] regionName) {
+    String encodedName;
+    if (hasEncodedName(regionName)) {
+      // region is in new format:
+      // <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
+      encodedName = Bytes.toString(regionName,
+      regionName.length - MD5_HEX_LENGTH - 1,
+      MD5_HEX_LENGTH);
+    } else {
+      // old format region name. First hbase:meta region also
+      // use this format.EncodedName is the JenkinsHash value.
+      HashKey<byte[]> key = new ByteArrayHashKey(regionName, 0, regionName.length);
+      int hashVal = Math.abs(JenkinsHash.getInstance().hash(key, 0));
+      encodedName = String.valueOf(hashVal);
+    }
+    return encodedName;
+  }
+
+  /**
+   * @return Return a String of short, printable names for <code>hris</code>
+   * (usually encoded name) for us logging.
+   */
+  static String getShortNameToLog(RegionInfo...hris) {
+    return getShortNameToLog(Arrays.asList(hris));
+  }
+
+  /**
+   * @return Return a String of short, printable names for <code>hris</code>
+   * (usually encoded name) for us logging.
+   */
+  static String getShortNameToLog(final List<RegionInfo> ris) {
+    return ris.stream().map(ri -> ri.getShortNameToLog()).
+    collect(Collectors.toList()).toString();
+  }
+
+  /**
+   * Gets the table name from the specified region name.
+   * @param regionName to extract the table name from
+   * @return Table name
+   */
+  @InterfaceAudience.Private
+  // This method should never be used. Its awful doing parse from bytes.
+  // It is fallback in case we can't get the tablename any other way. Could try removing it.
+  // Keeping it Audience Private so can remove at later date.
+  static TableName getTable(final byte [] regionName) {
+    int offset = -1;
+    for (int i = 0; i < regionName.length; i++) {
+      if (regionName[i] == HConstants.DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    byte[] buff  = new byte[offset];
+    System.arraycopy(regionName, 0, buff, 0, offset);
+    return TableName.valueOf(buff);
+  }
+
+  /**
+   * Gets the start key from the specified region name.
+   * @param regionName
+   * @return Start key.
+   * @throws java.io.IOException
+   */
+  static byte[] getStartKey(final byte[] regionName) throws IOException {
+    return parseRegionName(regionName)[1];
+  }
+
+  @InterfaceAudience.Private
+  static boolean isEncodedRegionName(byte[] regionName) throws IOException {
+    try {
+      parseRegionName(regionName);
+      return false;
+    } catch (IOException e) {
+      if (StringUtils.stringifyException(e)
+      .contains(INVALID_REGION_NAME_FORMAT_MESSAGE)) {
+        return true;
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * @param bytes
+   * @return A deserialized {@link RegionInfo}
+   * or null if we failed deserialize or passed bytes null
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFromOrNull(final byte [] bytes) {
+    if (bytes == null) return null;
+    return parseFromOrNull(bytes, 0, bytes.length);
+  }
+
+  /**
+   * @param bytes
+   * @param offset
+   * @param len
+   * @return A deserialized {@link RegionInfo} or null
+   *  if we failed deserialize or passed bytes null
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFromOrNull(final byte [] bytes, int offset, int len) {
+    if (bytes == null || len <= 0) return null;
+    try {
+      return parseFrom(bytes, offset, len);
+    } catch (DeserializationException e) {
+      return null;
+    }
+  }
+
+  /**
+   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
+   * @return A deserialized {@link RegionInfo}
+   * @throws DeserializationException
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
+    if (bytes == null) return null;
+    return parseFrom(bytes, 0, bytes.length);
+  }
+
+  /**
+   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
+   * @param offset starting point in the byte array
+   * @param len length to read on the byte array
+   * @return A deserialized {@link RegionInfo}
+   * @throws DeserializationException
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFrom(final byte [] bytes, int offset, int len)
+  throws DeserializationException {
+    if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) {
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      try {
+        HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
+        ProtobufUtil.mergeFrom(builder, bytes, pblen + offset, len - pblen);
+        HBaseProtos.RegionInfo ri = builder.build();
+        return ProtobufUtil.toRegionInfo(ri);
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+    } else {
+      throw new DeserializationException("PB encoded RegionInfo expected");
+    }
+  }
+
+  /**
+   * Check whether two regions are adjacent
+   * @param regionA
+   * @param regionB
+   * @return true if two regions are adjacent
+   */
+  static boolean areAdjacent(RegionInfo regionA, RegionInfo regionB) {
+    if (regionA == null || regionB == null) {
+      throw new IllegalArgumentException(
+      "Can't check whether adjacent for null region");
+    }
+    RegionInfo a = regionA;
+    RegionInfo b = regionB;
+    if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) {
+      a = regionB;
+      b = regionA;
+    }
+    if (Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * @param ri
+   * @return This instance serialized as protobuf w/ a magic pb prefix.
+   * @see #parseFrom(byte[])
+   */
+  static byte [] toByteArray(RegionInfo ri) {
+    byte [] bytes = ProtobufUtil.toProtoRegionInfo(ri).toByteArray();
+    return ProtobufUtil.prependPBMagic(bytes);
+  }
+
+  /**
+   * Use logging.
+   * @param encodedRegionName The encoded regionname.
+   * @return <code>hbase:meta</code> if passed <code>1028785192</code> else returns
+   * <code>encodedRegionName</code>
+   */
+  static String prettyPrint(final String encodedRegionName) {
+    if (encodedRegionName.equals("1028785192")) {
+      return encodedRegionName + "/hbase:meta";
+    }
+    return encodedRegionName;
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param regionid Region id (Usually timestamp from when region was created).
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  static byte [] createRegionName(final TableName tableName, final byte[] startKey,
+                                  final long regionid, boolean newFormat) {
+    return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id (Usually timestamp from when region was created).
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  static byte [] createRegionName(final TableName tableName,
+                                  final byte[] startKey, final String id, boolean newFormat) {
+    return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param regionid Region id (Usually timestamp from when region was created).
+   * @param replicaId
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey, id and replicaId
+   */
+  static byte [] createRegionName(final TableName tableName,
+      final byte[] startKey, final long regionid, int replicaId, boolean newFormat) {
+    return createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
+      replicaId, newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id (Usually timestamp from when region was created).
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  static byte [] createRegionName(final TableName tableName,
+      final byte[] startKey, final byte[] id, boolean newFormat) {
+    return createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id (Usually timestamp from when region was created).
+   * @param replicaId
+   * @param newFormat should we create the region name in the new format
+   * @return Region name made of passed tableName, startKey, id and replicaId
+   */
+  static byte [] createRegionName(final TableName tableName,
+      final byte[] startKey, final byte[] id, final int replicaId, boolean newFormat) {
+    int len = tableName.getName().length + 2 + id.length + (startKey == null? 0: startKey.length);
+    if (newFormat) {
+      len += MD5_HEX_LENGTH + 2;
+    }
+    byte[] replicaIdBytes = null;
+    // Special casing: replicaId is only appended if replicaId is greater than
+    // 0. This is because all regions in meta would have to be migrated to the new
+    // name otherwise
+    if (replicaId > 0) {
+      // use string representation for replica id
+      replicaIdBytes = Bytes.toBytes(String.format(REPLICA_ID_FORMAT, replicaId));
+      len += 1 + replicaIdBytes.length;
+    }
+
+    byte [] b = new byte [len];
+
+    int offset = tableName.getName().length;
+    System.arraycopy(tableName.getName(), 0, b, 0, offset);
+    b[offset++] = HConstants.DELIMITER;
+    if (startKey != null && startKey.length > 0) {
+      System.arraycopy(startKey, 0, b, offset, startKey.length);
+      offset += startKey.length;
+    }
+    b[offset++] = HConstants.DELIMITER;
+    System.arraycopy(id, 0, b, offset, id.length);
+    offset += id.length;
+
+    if (replicaIdBytes != null) {
+      b[offset++] = REPLICA_ID_DELIMITER;
+      System.arraycopy(replicaIdBytes, 0, b, offset, replicaIdBytes.length);
+      offset += replicaIdBytes.length;
+    }
+
+    if (newFormat) {
+      //
+      // Encoded name should be built into the region name.
+      //
+      // Use the region name thus far (namely, <tablename>,<startKey>,<id>_<replicaId>)
+      // to compute a MD5 hash to be used as the encoded name, and append
+      // it to the byte buffer.
+      //
+      String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
+      byte [] md5HashBytes = Bytes.toBytes(md5Hash);
+
+      if (md5HashBytes.length != MD5_HEX_LENGTH) {
+        System.out.println("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
+        "; Got=" + md5HashBytes.length);
+      }
+
+      // now append the bytes '.<encodedName>.' to the end
+      b[offset++] = ENC_SEPARATOR;
+      System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
+      offset += MD5_HEX_LENGTH;
+      b[offset++] = ENC_SEPARATOR;
+    }
+
+    return b;
+  }
+
+  /**
+   * Separate elements of a regionName.
+   * @param regionName
+   * @return Array of byte[] containing tableName, startKey and id
+   * @throws IOException
+   */
+  static byte [][] parseRegionName(final byte[] regionName)
+  throws IOException {
+    // Region name is of the format:
+    // tablename,startkey,regionIdTimestamp[_replicaId][.encodedName.]
+    // startkey can contain the delimiter (',') so we parse from the start and end
+
+    // parse from start
+    int offset = -1;
+    for (int i = 0; i < regionName.length; i++) {
+      if (regionName[i] == HConstants.DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    if (offset == -1) {
+      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
+      + ": " + Bytes.toStringBinary(regionName));
+    }
+    byte[] tableName = new byte[offset];
+    System.arraycopy(regionName, 0, tableName, 0, offset);
+    offset = -1;
+
+    int endOffset = regionName.length;
+    // check whether regionName contains encodedName
+    if (regionName.length > MD5_HEX_LENGTH + 2
+    && regionName[regionName.length-1] == ENC_SEPARATOR
+    && regionName[regionName.length-MD5_HEX_LENGTH-2] == ENC_SEPARATOR) {
+      endOffset = endOffset - MD5_HEX_LENGTH - 2;
+    }
+
+    // parse from end
+    byte[] replicaId = null;
+    int idEndOffset = endOffset;
+    for (int i = endOffset - 1; i > 0; i--) {
+      if (regionName[i] == REPLICA_ID_DELIMITER) { //replicaId may or may not be present
+        replicaId = new byte[endOffset - i - 1];
+        System.arraycopy(regionName, i + 1, replicaId, 0,
+        endOffset - i - 1);
+        idEndOffset = i;
+        // do not break, continue to search for id
+      }
+      if (regionName[i] == HConstants.DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    if (offset == -1) {
+      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
+      + ": " + Bytes.toStringBinary(regionName));
+    }
+    byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
+    if(offset != tableName.length + 1) {
+      startKey = new byte[offset - tableName.length - 1];
+      System.arraycopy(regionName, tableName.length + 1, startKey, 0,
+      offset - tableName.length - 1);
+    }
+    byte [] id = new byte[idEndOffset - offset - 1];
+    System.arraycopy(regionName, offset + 1, id, 0,
+    idEndOffset - offset - 1);
+    byte [][] elements = new byte[replicaId == null ? 3 : 4][];
+    elements[0] = tableName;
+    elements[1] = startKey;
+    elements[2] = id;
+    if (replicaId != null) {
+      elements[3] = replicaId;
+    }
+    return elements;
+  }
+
+  /**
+   * Serializes given RegionInfo's as a byte array. Use this instead of
+   * {@link RegionInfo#toByteArray(RegionInfo)} when
+   * writing to a stream and you want to use the pb mergeDelimitedFrom (w/o the delimiter, pb reads
+   * to EOF which may not be what you want). {@link #parseDelimitedFrom(byte[], int, int)} can
+   * be used to read back the instances.
+   * @param infos RegionInfo objects to serialize
+   * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
+   * @throws IOException
+   */
+  static byte[] toDelimitedByteArray(RegionInfo... infos) throws IOException {
+    byte[][] bytes = new byte[infos.length][];
+    int size = 0;
+    for (int i = 0; i < infos.length; i++) {
+      bytes[i] = toDelimitedByteArray(infos[i]);
+      size += bytes[i].length;
+    }
+
+    byte[] result = new byte[size];
+    int offset = 0;
+    for (byte[] b : bytes) {
+      System.arraycopy(b, 0, result, offset, b.length);
+      offset += b.length;
+    }
+    return result;
+  }
+
+  /**
+   * Use this instead of {@link RegionInfo#toByteArray(RegionInfo)} when writing to a stream and you want to use
+   * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want).
+   * @param ri
+   * @return This instance serialized as a delimied protobuf w/ a magic pb prefix.
+   * @throws IOException
+   */
+  static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException {
+    return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toProtoRegionInfo(ri));
+  }
+
+  /**
+   * Parses an RegionInfo instance from the passed in stream.
+   * Presumes the RegionInfo was serialized to the stream with
+   * {@link #toDelimitedByteArray(RegionInfo)}.
+   * @param in
+   * @return An instance of RegionInfo.
+   * @throws IOException
+   */
+  static RegionInfo parseFrom(final DataInputStream in) throws IOException {
+    // I need to be able to move back in the stream if this is not a pb
+    // serialization so I can do the Writable decoding instead.
+    int pblen = ProtobufUtil.lengthOfPBMagic();
+    byte [] pbuf = new byte[pblen];
+    if (in.markSupported()) { //read it with mark()
+      in.mark(pblen);
+    }
+
+    //assumption: if Writable serialization, it should be longer than pblen.
+    int read = in.read(pbuf);
+    if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
+    if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
+      return ProtobufUtil.toRegionInfo(HBaseProtos.RegionInfo.parseDelimitedFrom(in));
+    } else {
+      throw new IOException("PB encoded RegionInfo expected");
+    }
+  }
+
+  /**
+   * Parses all the RegionInfo instances from the passed in stream until EOF. Presumes the
+   * RegionInfo's were serialized to the stream with oDelimitedByteArray()
+   * @param bytes serialized bytes
+   * @param offset the start offset into the byte[] buffer
+   * @param length how far we should read into the byte[] buffer
+   * @return All the RegionInfos that are in the byte array. Keeps reading till we hit the end.
+   * @throws IOException
+   */
+  static List<RegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
+                                             final int length) throws IOException {
+    if (bytes == null) {
+      throw new IllegalArgumentException("Can't build an object with empty bytes array");
+    }
+    DataInputBuffer in = new DataInputBuffer();
+    List<RegionInfo> ris = new ArrayList<>();
+    try {
+      in.reset(bytes, offset, length);
+      while (in.available() > 0) {
+        RegionInfo ri = parseFrom(in);
+        ris.add(ri);
+      }
+    } finally {
+      in.close();
+    }
+    return ris;
+  }
+}
\ No newline at end of file


[13/47] hbase git commit: HBASE-17351 Enforcer plugin fails with NullPointerException

Posted by bu...@apache.org.
HBASE-17351 Enforcer plugin fails with NullPointerException


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

Branch: refs/heads/HBASE-18467
Commit: fb3c8bf6087b1d9e086a6becd1c0e518cb1f8936
Parents: 9c5b03a
Author: Michael Stack <st...@apache.org>
Authored: Thu Sep 14 17:20:59 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 14 17:22:41 2017 -0700

----------------------------------------------------------------------
 pom.xml | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fb3c8bf6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 29a2505..019501c 100755
--- a/pom.xml
+++ b/pom.xml
@@ -812,7 +812,7 @@
                   <pluginExecutionFilter>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-enforcer-plugin</artifactId>
-                    <versionRange>[1.0.1,)</versionRange>
+                    <version>${enforcer.version}</version>
                     <goals>
                       <goal>enforce</goal>
                     </goals>
@@ -971,6 +971,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
+        <version>${enforcer.version}</version>
         <dependencies>
           <dependency>
             <groupId>org.codehaus.mojo</groupId>
@@ -1517,6 +1518,8 @@
     <argLine>${hbase-surefire.argLine}</argLine>
     <jacoco.version>0.7.5.201505241946</jacoco.version>
     <extra.enforcer.version>1.0-beta-6</extra.enforcer.version>
+    <!--See HBASE-17351; we need to set version to 1.4. 1.4.1 fails with MENFORCER-248-->
+    <enforcer.version>1.4</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>
@@ -2314,6 +2317,7 @@
           <plugin>
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-enforcer-plugin</artifactId>
+            <version>${enforcer.version}</version>
             <configuration>
               <rules>
                 <enforceBytecodeVersion>


[17/47] hbase git commit: HBASE-18446 Mark StoreFileScanner/StoreFileReader as IA.LimitedPrivate(Phoenix)

Posted by bu...@apache.org.
HBASE-18446 Mark StoreFileScanner/StoreFileReader as IA.LimitedPrivate(Phoenix)


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

Branch: refs/heads/HBASE-18467
Commit: a6d8cedb06eecbdb1d15e8067d470f5b871187c1
Parents: 4341c3f
Author: zhangduo <zh...@apache.org>
Authored: Fri Sep 15 16:35:21 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Sep 15 20:36:46 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/coprocessor/RegionObserver.java  |  7 +++++--
 .../hadoop/hbase/regionserver/StoreFileReader.java       | 11 +++++++----
 .../hadoop/hbase/regionserver/StoreFileScanner.java      |  5 ++++-
 3 files changed, 16 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a6d8cedb/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 ae57747..9f16c7c 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
@@ -239,8 +239,7 @@ public interface RegionObserver extends Coprocessor {
    * effect in this hook.
    * @param c the environment provided by the region server
    * @param store the store being compacted
-   * @param scanners the list {@link org.apache.hadoop.hbase.regionserver.StoreFileScanner}s
-   *  to be read from
+   * @param scanners the list of store file scanners to be read from
    * @param scanType the {@link ScanType} indicating whether this is a major or minor compaction
    * @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
    *          files
@@ -1034,7 +1033,9 @@ public interface RegionObserver extends Coprocessor {
    * @param reader the base reader, if not {@code null}, from previous RegionObserver in the chain
    * @return a Reader instance to use instead of the base reader if overriding
    * default behavior, null otherwise
+   * @deprecated For Phoenix only, StoreFileReader is not a stable interface.
    */
+  @Deprecated
   default StoreFileReader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
       FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
       Reference r, StoreFileReader reader) throws IOException {
@@ -1053,7 +1054,9 @@ public interface RegionObserver extends Coprocessor {
    * @param r original reference file. This will be not null only when reading a split file.
    * @param reader the base reader instance
    * @return The reader to use
+   * @deprecated For Phoenix only, StoreFileReader is not a stable interface.
    */
+  @Deprecated
   default StoreFileReader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
       FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
       Reference r, StoreFileReader reader) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6d8cedb/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
index 06f42c8..e3f97a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.Map;
@@ -34,9 +32,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.TimeRange;
@@ -49,11 +47,16 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.util.BloomFilter;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * Reader for a StoreFile.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.PHOENIX)
+@InterfaceStability.Evolving
 public class StoreFileReader {
   private static final Log LOG = LogFactory.getLog(StoreFileReader.class.getName());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a6d8cedb/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
index e1d4c87..08111dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
@@ -32,8 +32,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@@ -43,7 +45,8 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
  * KeyValueScanner adaptor over the Reader.  It also provides hooks into
  * bloom filter things.
  */
-@InterfaceAudience.LimitedPrivate("Coprocessor")
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.PHOENIX)
+@InterfaceStability.Evolving
 public class StoreFileScanner implements KeyValueScanner {
   // the reader it comes from:
   private final StoreFileReader reader;