You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/06/17 05:01:43 UTC

[01/30] hbase git commit: HBASE-14644 Region in transition metric is broken (Huaxiang Sun)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 bd45cf347 -> 3abd52bdc


HBASE-14644 Region in transition metric is broken (Huaxiang Sun)


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

Branch: refs/heads/hbase-12439
Commit: 4c885f4fdbef1efa7f4b6f43bc024db6280cde39
Parents: bd45cf3
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Fri Jun 10 21:49:10 2016 -0700
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Jun 10 21:52:24 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/master/AssignmentManager.java  |   4 +
 .../org/apache/hadoop/hbase/master/HMaster.java |  26 +++-
 .../hbase/master/MetricsAssignmentManager.java  |   4 +
 .../hbase/regionserver/HRegionServer.java       |   4 -
 .../master/TestAssignmentManagerMetrics.java    | 136 +++++++++++++++++++
 5 files changed, 168 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4c885f4f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index f16463f..49026ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -258,6 +258,10 @@ public class AssignmentManager {
     this.tableLockManager = tableLockManager;
   }
 
+  MetricsAssignmentManager getAssignmentManagerMetrics() {
+    return this.metricsAssignmentManager;
+  }
+
   /**
    * Add the listener to the notification list.
    * @param listener The AssignmentListener to register

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c885f4f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index d368ffb..efb6b6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.RegionStateListener;
+import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
@@ -305,6 +306,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   private RegionNormalizerChore normalizerChore;
   private ClusterStatusChore clusterStatusChore;
   private ClusterStatusPublisher clusterStatusPublisherChore = null;
+  private PeriodicDoMetrics periodicDoMetricsChore = null;
 
   CatalogJanitor catalogJanitorChore;
   private LogCleaner logCleaner;
@@ -370,6 +372,19 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
+  private static class PeriodicDoMetrics extends ScheduledChore {
+    private final HMaster server;
+    public PeriodicDoMetrics(int doMetricsInterval, final HMaster server) {
+      super(server.getServerName() + "-DoMetricsChore", server, doMetricsInterval);
+      this.server = server;
+    }
+
+    @Override
+    protected void chore() {
+      server.doMetrics();
+    }
+  }
+
   /**
    * Initializes the HMaster. The steps are as follows:
    * <p>
@@ -433,6 +448,10 @@ public class HMaster extends HRegionServer implements MasterServices {
       }
     }
 
+    // Do Metrics periodically
+    periodicDoMetricsChore = new PeriodicDoMetrics(msgInterval, this);
+    getChoreService().scheduleChore(periodicDoMetricsChore);
+
     // Some unit tests don't need a cluster, so no zookeeper at all
     if (!conf.getBoolean("hbase.testing.nocluster", false)) {
       activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
@@ -563,8 +582,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * Emit the HMaster metrics, such as region in transition metrics.
    * Surrounding in a try block just to be sure metrics doesn't abort HMaster.
    */
-  @Override
-  protected void doMetrics() {
+  private void doMetrics() {
     try {
       if (assignmentManager != null) {
         assignmentManager.updateRegionsInTransitionMetrics();
@@ -1202,6 +1220,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.mobCompactThread != null) {
       this.mobCompactThread.close();
     }
+
+    if (this.periodicDoMetricsChore != null) {
+      periodicDoMetricsChore.cancel();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c885f4f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
index 7b2423c..b89d2da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
@@ -29,6 +29,10 @@ public class MetricsAssignmentManager {
         MetricsAssignmentManagerSource.class);
   }
 
+  public MetricsAssignmentManagerSource getMetricsProcSource() {
+    return assignmentManagerSource;
+  }
+
   public void updateAssignmentTime(long time) {
     assignmentManagerSource.updateAssignmentTime(time);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c885f4f/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 279affa..396e182 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
@@ -688,9 +688,6 @@ public class HRegionServer extends HasThread implements
     return RSDumpServlet.class;
   }
 
-  protected void doMetrics() {
-  }
-
   @Override
   public boolean registerService(Service instance) {
     /*
@@ -994,7 +991,6 @@ public class HRegionServer extends HasThread implements
         if ((now - lastMsg) >= msgInterval) {
           tryRegionServerReport(lastMsg, now);
           lastMsg = System.currentTimeMillis();
-          doMetrics();
         }
         if (!isStopped() && !isAborted()) {
           this.sleeper.sleep();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4c885f4f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java
new file mode 100644
index 0000000..4817457
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerMetrics.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompatibilityFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+
+@Category(MediumTests.class)
+public class TestAssignmentManagerMetrics {
+
+  private static final Log LOG = LogFactory.getLog(TestAssignmentManagerMetrics.class);
+  private static final MetricsAssertHelper metricsHelper = CompatibilityFactory
+      .getInstance(MetricsAssertHelper.class);
+
+  private static MiniHBaseCluster cluster;
+  private static HMaster master;
+  private static HBaseTestingUtility TEST_UTIL;
+  private static Configuration conf;
+  private static final int msgInterval = 1000;
+
+  @BeforeClass
+  public static void startCluster() throws Exception {
+    LOG.info("Starting cluster");
+    TEST_UTIL = new HBaseTestingUtility();
+    conf = TEST_UTIL.getConfiguration();
+
+    // Disable sanity check for coprocessor
+    conf.setBoolean("hbase.table.sanity.checks", false);
+
+    // set RIT stuck warning threshold to a small value
+    conf.setInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 20);
+
+    // set msgInterval to 1 second
+    conf.setInt("hbase.regionserver.msginterval", msgInterval);
+
+    // set tablesOnMaster to none
+    conf.set("hbase.balancer.tablesOnMaster", "none");
+
+    TEST_UTIL.startMiniCluster(1);
+    cluster = TEST_UTIL.getHBaseCluster();
+    master = cluster.getMaster();
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    if (TEST_UTIL != null) {
+      TEST_UTIL.shutdownMiniCluster();
+    }
+  }
+
+  @Test
+  public void testRITAssignmentManagerMetrics() throws Exception {
+
+    final TableName TABLENAME = TableName.valueOf("testRITMetrics");
+    final byte[] FAMILY = Bytes.toBytes("family");
+
+    Table table = null;
+    try {
+      table = TEST_UTIL.createTable(TABLENAME, FAMILY);
+
+      final byte[] row = Bytes.toBytes("row");
+      final byte[] qualifier = Bytes.toBytes("qualifier");
+      final byte[] value = Bytes.toBytes("value");
+
+      Put put = new Put(row);
+      put.addColumn(FAMILY, qualifier, value);
+      table.put(put);
+
+      // Sleep 3 seconds, wait for doMetrics chore catching up
+      Thread.sleep(msgInterval * 3);
+
+      // check the RIT is 0
+      MetricsAssignmentManagerSource amSource =
+          master.getAssignmentManager().getAssignmentManagerMetrics().getMetricsProcSource();
+
+      metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_NAME, 0, amSource);
+      metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_OVER_THRESHOLD_NAME, 0,
+          amSource);
+
+      // alter table with a non-existing coprocessor
+      HTableDescriptor htd = new HTableDescriptor(TABLENAME);
+      HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
+
+      htd.addFamily(hcd);
+
+      String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
+      htd.addCoprocessorWithSpec(spec);
+
+      TEST_UTIL.getHBaseAdmin().modifyTable(TABLENAME, htd);
+
+      // Sleep 3 seconds, wait for doMetrics chore catching up
+      Thread.sleep(msgInterval * 3);
+      metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_NAME, 1, amSource);
+      metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_OVER_THRESHOLD_NAME, 1,
+          amSource);
+
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+}


[21/30] hbase git commit: HBASE-5291 Addendum removes duplicate spnego (Josh Elser)

Posted by sy...@apache.org.
HBASE-5291 Addendum removes duplicate spnego (Josh Elser)


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

Branch: refs/heads/hbase-12439
Commit: 1bad166f677ef565607c1f9660114a7a55c27b7b
Parents: bff35d6
Author: tedyu <yu...@gmail.com>
Authored: Wed Jun 15 14:59:17 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jun 15 14:59:17 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/1bad166f/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
index e8f875e..70b5242 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -110,7 +110,7 @@ public class HttpServer implements FilterContainer {
   static final String HTTP_MAX_THREADS = "hbase.http.max.threads";
 
   public static final String HTTP_UI_AUTHENTICATION = "hbase.security.authentication.ui";
-  static final String HTTP_AUTHENTICATION_PREFIX = "hbase.security.authentication.spnego.";
+  static final String HTTP_AUTHENTICATION_PREFIX = "hbase.security.authentication.";
   static final String HTTP_SPNEGO_AUTHENTICATION_PREFIX = HTTP_AUTHENTICATION_PREFIX
       + "spnego.";
   static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX = "kerberos.principal";


[24/30] hbase git commit: HBASE-15999 NPE in MemstoreCompactor (Ram)

Posted by sy...@apache.org.
HBASE-15999 NPE in MemstoreCompactor (Ram)


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

Branch: refs/heads/hbase-12439
Commit: f19f1d9e99c8058f6ef5caa42c1b07a8ae1de53f
Parents: 158568e
Author: Ramkrishna <ra...@intel.com>
Authored: Thu Jun 16 10:06:40 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Thu Jun 16 10:06:40 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/CompactingMemStore.java     | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f19f1d9e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
index d47ac36..ec5684d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
@@ -253,10 +253,11 @@ public class CompactingMemStore extends AbstractMemStore {
       * in exclusive mode while this method (checkActiveSize) is invoked holding updatesLock
       * in the shared mode. */
       InMemoryFlushRunnable runnable = new InMemoryFlushRunnable();
-      LOG.info("Dispatching the MemStore in-memory flush for store " + store.getColumnFamilyName());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(
+          "Dispatching the MemStore in-memory flush for store " + store.getColumnFamilyName());
+      }
       getPool().execute(runnable);
-      // guard against queuing same old compactions over and over again
-      inMemoryFlushInProgress.set(true);
     }
   }
 
@@ -277,10 +278,9 @@ public class CompactingMemStore extends AbstractMemStore {
     }
     // Phase II: Compact the pipeline
     try {
-      if (allowCompaction.get()) {
+      if (allowCompaction.get() && inMemoryFlushInProgress.compareAndSet(false, true)) {
         // setting the inMemoryFlushInProgress flag again for the case this method is invoked
         // directly (only in tests) in the common path setting from true to true is idempotent
-        inMemoryFlushInProgress.set(true);
         // Speculative compaction execution, may be interrupted if flush is forced while
         // compaction is in progress
         compactor.startCompaction();


[17/30] hbase git commit: HBASE-15974 Create a ReplicationQueuesClientHBaseImpl

Posted by sy...@apache.org.
HBASE-15974 Create a ReplicationQueuesClientHBaseImpl

Building on HBase-15958.
Provided a ReplicationQueuesClientHBaseImpl that relies on the HBase Replication Table to track WAL queues.
Refactored out a large section of ReplicationQueuesHBaseImpl into a ReplicationTableClient class that handles Replication Table operations.

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


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

Branch: refs/heads/hbase-12439
Commit: 2093aadec1433c1376d1a9a6166e94a257a811b2
Parents: ae5fe1e
Author: Joseph Hwang <jz...@fb.com>
Authored: Thu Jun 9 16:16:38 2016 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Jun 15 10:43:14 2016 -0700

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    |   4 +-
 .../hbase/replication/ReplicationFactory.java   |   9 +-
 .../replication/ReplicationQueuesClient.java    |  10 +-
 .../ReplicationQueuesClientArguments.java       |  35 +
 .../ReplicationQueuesClientZKImpl.java          |  51 +-
 .../replication/ReplicationQueuesHBaseImpl.java | 644 -------------------
 .../hbase/replication/ReplicationTableBase.java | 351 ++++++++++
 .../TableBasedReplicationQueuesClientImpl.java  | 111 ++++
 .../TableBasedReplicationQueuesImpl.java        | 437 +++++++++++++
 .../master/ReplicationHFileCleaner.java         |   8 +-
 .../master/ReplicationLogCleaner.java           |  46 +-
 .../hbase/util/hbck/ReplicationChecker.java     |   6 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |   3 +-
 .../TestReplicationStateHBaseImpl.java          |  90 ++-
 .../replication/TestReplicationStateZKImpl.java |   6 +-
 .../TestReplicationSourceManager.java           |   7 +-
 16 files changed, 1101 insertions(+), 717 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index d062448..e0985bd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
@@ -122,7 +123,8 @@ public class ReplicationAdmin implements Closeable {
       zkw = createZooKeeperWatcher();
       try {
         this.replicationQueuesClient =
-            ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
+            ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf,
+            this.connection, zkw));
         this.replicationQueuesClient.init();
         this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf,
           this.replicationQueuesClient, this.connection);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index e264a4d..38f9f30 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -38,9 +38,12 @@ public class ReplicationFactory {
     return (ReplicationQueues) ConstructorUtils.invokeConstructor(classToBuild, args);
   }
 
-  public static ReplicationQueuesClient getReplicationQueuesClient(final ZooKeeperWatcher zk,
-      Configuration conf, Abortable abortable) {
-    return new ReplicationQueuesClientZKImpl(zk, conf, abortable);
+  public static ReplicationQueuesClient getReplicationQueuesClient(
+      ReplicationQueuesClientArguments args)
+    throws Exception {
+    Class<?> classToBuild = args.getConf().getClass("hbase.region.replica." +
+      "replication.ReplicationQueuesClientType", ReplicationQueuesClientZKImpl.class);
+    return (ReplicationQueuesClient) ConstructorUtils.invokeConstructor(classToBuild, args);
   }
 
   public static ReplicationPeers getReplicationPeers(final ZooKeeperWatcher zk, Configuration conf,

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
index 7fa3bbb..6d8900e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
@@ -61,11 +62,12 @@ public interface ReplicationQueuesClient {
   List<String> getAllQueues(String serverName) throws KeeperException;
 
   /**
-   * Get the cversion of replication rs node. This can be used as optimistic locking to get a
-   * consistent snapshot of the replication queues.
-   * @return cversion of replication rs node
+   * Load all wals in all replication queues from ZK. This method guarantees to return a
+   * snapshot which contains all WALs in the zookeeper at the start of this call even there
+   * is concurrent queue failover. However, some newly created WALs during the call may
+   * not be included.
    */
-  int getQueuesZNodeCversion() throws KeeperException;
+   Set<String> getAllWALs() throws KeeperException;
 
   /**
    * Get the change version number of replication hfile references node. This can be used as

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
new file mode 100644
index 0000000..8a61993
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
@@ -0,0 +1,35 @@
+/*
+ *
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
+@InterfaceAudience.Private
+public class ReplicationQueuesClientArguments extends ReplicationQueuesArguments {
+  public ReplicationQueuesClientArguments(Configuration conf, Abortable abort,
+     ZooKeeperWatcher zk) {
+    super(conf, abort, zk);
+  }
+  public ReplicationQueuesClientArguments(Configuration conf, Abortable abort) {
+    super(conf, abort);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index cc407e3..b0ded7d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -19,7 +19,12 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
+import java.util.Set;
 
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
@@ -32,6 +37,12 @@ import org.apache.zookeeper.data.Stat;
 public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements
     ReplicationQueuesClient {
 
+  Log LOG = LogFactory.getLog(ReplicationQueuesClientZKImpl.class);
+
+  public ReplicationQueuesClientZKImpl(ReplicationQueuesClientArguments args) {
+    this(args.getZk(), args.getConf(), args.getAbortable());
+  }
+
   public ReplicationQueuesClientZKImpl(final ZooKeeperWatcher zk, Configuration conf,
       Abortable abortable) {
     super(zk, conf, abortable);
@@ -74,7 +85,45 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
     return result;
   }
 
-  @Override public int getQueuesZNodeCversion() throws KeeperException {
+  @Override
+  public Set<String> getAllWALs() throws KeeperException {
+    /**
+     * Load all wals in all replication queues from ZK. This method guarantees to return a
+     * snapshot which contains all WALs in the zookeeper at the start of this call even there
+     * is concurrent queue failover. However, some newly created WALs during the call may
+     * not be included.
+     */
+    for (int retry = 0; ; retry++) {
+      int v0 = getQueuesZNodeCversion();
+      List<String> rss = getListOfReplicators();
+      if (rss == null) {
+        LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
+        return ImmutableSet.of();
+      }
+      Set<String> wals = Sets.newHashSet();
+      for (String rs : rss) {
+        List<String> listOfPeers = getAllQueues(rs);
+        // if rs just died, this will be null
+        if (listOfPeers == null) {
+          continue;
+        }
+        for (String id : listOfPeers) {
+          List<String> peersWals = getLogsInQueue(rs, id);
+          if (peersWals != null) {
+            wals.addAll(peersWals);
+          }
+        }
+      }
+      int v1 = getQueuesZNodeCversion();
+      if (v0 == v1) {
+        return wals;
+      }
+      LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
+        v0, v1, retry));
+    }
+  }
+
+  public int getQueuesZNodeCversion() throws KeeperException {
     try {
       Stat stat = new Stat();
       ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesHBaseImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesHBaseImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesHBaseImpl.java
deleted file mode 100644
index 34a5289..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesHBaseImpl.java
+++ /dev/null
@@ -1,644 +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.replication;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.CompareFilter;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.RetryCounterFactory;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-import sun.reflect.generics.reflectiveObjects.NotImplementedException;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * This class provides an implementation of the ReplicationQueues interface using an HBase table
- * "Replication Table". The basic schema of this table will store each individual queue as a
- * seperate row. The row key will be a unique identifier of the creating server's name and the
- * queueId. Each queue must have the following two columns:
- *  COL_OWNER: tracks which server is currently responsible for tracking the queue
- *  COL_QUEUE_ID: tracks the queue's id as stored in ReplicationSource
- * They will also have columns mapping [WAL filename : offset]
- * One key difference from the ReplicationQueuesZkImpl is that when queues are reclaimed we
- * simply return its HBase row key as its new "queueId"
- */
-
-@InterfaceAudience.Private
-public class ReplicationQueuesHBaseImpl extends ReplicationStateZKBase
-    implements ReplicationQueues {
-
-  private static final Log LOG = LogFactory.getLog(ReplicationQueuesHBaseImpl.class);
-
-  /** Name of the HBase Table used for tracking replication*/
-  public static final TableName REPLICATION_TABLE_NAME =
-    TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication");
-
-  // Column family and column names for the Replication Table
-  private static final byte[] CF = Bytes.toBytes("r");
-  private static final byte[] COL_OWNER = Bytes.toBytes("o");
-  private static final byte[] COL_OWNER_HISTORY = Bytes.toBytes("h");
-
-  // The value used to delimit the queueId and server name inside of a queue's row key. Currently a
-  // hyphen, because it is guaranteed that queueId (which is a cluster id) cannot contain hyphens.
-  // See HBASE-11394.
-  private static String ROW_KEY_DELIMITER = "-";
-
-  // Column Descriptor for the Replication Table
-  private static final HColumnDescriptor REPLICATION_COL_DESCRIPTOR =
-    new HColumnDescriptor(CF).setMaxVersions(1)
-      .setInMemory(true)
-      .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-        // TODO: Figure out which bloom filter to use
-      .setBloomFilterType(BloomType.NONE)
-      .setCacheDataInL1(true);
-
-  // Common byte values used in replication offset tracking
-  private static final byte[] INITIAL_OFFSET_BYTES = Bytes.toBytes(0L);
-  private static final byte[] EMPTY_STRING_BYTES = Bytes.toBytes("");
-
-  /*
-   * Make sure that HBase table operations for replication have a high number of retries. This is
-   * because the server is aborted if any HBase table operation fails. Each RPC will be attempted
-   * 3600 times before exiting. This provides each operation with 2 hours of retries
-   * before the server is aborted.
-   */
-  private static final int CLIENT_RETRIES = 3600;
-  private static final int RPC_TIMEOUT = 2000;
-  private static final int OPERATION_TIMEOUT = CLIENT_RETRIES * RPC_TIMEOUT;
-
-  private Configuration modifiedConf;
-  private Admin admin;
-  private Connection connection;
-  private Table replicationTable;
-  private String serverName = null;
-  private byte[] serverNameBytes = null;
-
-  public ReplicationQueuesHBaseImpl(ReplicationQueuesArguments args) {
-    this(args.getConf(), args.getAbortable(), args.getZk());
-  }
-
-  public ReplicationQueuesHBaseImpl(Configuration conf, Abortable abort, ZooKeeperWatcher zkw) {
-    super(zkw, conf, abort);
-    modifiedConf = new Configuration(conf);
-    modifiedConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES);
-  }
-
-  @Override
-  public void init(String serverName) throws ReplicationException {
-    try {
-      this.serverName = serverName;
-      this.serverNameBytes = Bytes.toBytes(serverName);
-      // Modify the connection's config so that the Replication Table it returns has a much higher
-      // number of client retries
-      this.connection = ConnectionFactory.createConnection(modifiedConf);
-      this.admin = connection.getAdmin();
-      replicationTable = createAndGetReplicationTable();
-      replicationTable.setRpcTimeout(RPC_TIMEOUT);
-      replicationTable.setOperationTimeout(OPERATION_TIMEOUT);
-    } catch (IOException e) {
-      throw new ReplicationException(e);
-    }
-  }
-
-  @Override
-  public void removeQueue(String queueId) {
-
-    try {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      Delete deleteQueue = new Delete(rowKey);
-      safeQueueUpdate(deleteQueue);
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed removing queue queueId=" + queueId;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public void addLog(String queueId, String filename) throws ReplicationException {
-    try {
-      if (!checkQueueExists(queueId)) {
-        // Each queue will have an Owner, OwnerHistory, and a collection of [WAL:offset] key values
-        Put putNewQueue = new Put(Bytes.toBytes(buildQueueRowKey(queueId)));
-        putNewQueue.addColumn(CF, COL_OWNER, serverNameBytes);
-        putNewQueue.addColumn(CF, COL_OWNER_HISTORY, EMPTY_STRING_BYTES);
-        putNewQueue.addColumn(CF, Bytes.toBytes(filename), INITIAL_OFFSET_BYTES);
-        replicationTable.put(putNewQueue);
-      } else {
-        // Otherwise simply add the new log and offset as a new column
-        Put putNewLog = new Put(queueIdToRowKey(queueId));
-        putNewLog.addColumn(CF, Bytes.toBytes(filename), INITIAL_OFFSET_BYTES);
-        safeQueueUpdate(putNewLog);
-      }
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed adding log queueId=" + queueId + " filename=" + filename;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public void removeLog(String queueId, String filename) {
-    try {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      Delete delete = new Delete(rowKey);
-      delete.addColumns(CF, Bytes.toBytes(filename));
-      safeQueueUpdate(delete);
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed removing log queueId=" + queueId + " filename=" + filename;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public void setLogPosition(String queueId, String filename, long position) {
-    try {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      // Check that the log exists. addLog() must have been called before setLogPosition().
-      Get checkLogExists = new Get(rowKey);
-      checkLogExists.addColumn(CF, Bytes.toBytes(filename));
-      if (!replicationTable.exists(checkLogExists)) {
-        String errMsg = "Could not set position of non-existent log from queueId=" + queueId +
-          ", filename=" + filename;
-        abortable.abort(errMsg, new ReplicationException(errMsg));
-        return;
-      }
-      // Update the log offset if it exists
-      Put walAndOffset = new Put(rowKey);
-      walAndOffset.addColumn(CF, Bytes.toBytes(filename), Bytes.toBytes(position));
-      safeQueueUpdate(walAndOffset);
-    } catch (IOException | ReplicationException e) {
-      String errMsg = "Failed writing log position queueId=" + queueId + "filename=" +
-        filename + " position=" + position;
-      abortable.abort(errMsg, e);
-    }
-  }
-
-  @Override
-  public long getLogPosition(String queueId, String filename) throws ReplicationException {
-    try {
-      byte[] rowKey = queueIdToRowKey(queueId);
-      Get getOffset = new Get(rowKey);
-      getOffset.addColumn(CF, Bytes.toBytes(filename));
-      Result result = getResultIfOwner(getOffset);
-      if (result == null || !result.containsColumn(CF, Bytes.toBytes(filename))) {
-        throw new ReplicationException("Could not read empty result while getting log position " +
-            "queueId=" + queueId + ", filename=" + filename);
-      }
-      return Bytes.toLong(result.getValue(CF, Bytes.toBytes(filename)));
-    } catch (IOException e) {
-      throw new ReplicationException("Could not get position in log for queueId=" + queueId +
-          ", filename=" + filename);
-    }
-  }
-
-  @Override
-  public void removeAllQueues() {
-    List<String> myQueueIds = getAllQueues();
-    for (String queueId : myQueueIds) {
-      removeQueue(queueId);
-    }
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String queueId) {
-    byte[] rowKey = queueIdToRowKey(queueId);
-    return getLogsInQueue(rowKey);
-  }
-
-  private List<String> getLogsInQueue(byte[] rowKey) {
-    String errMsg = "Could not get logs in queue queueId=" + Bytes.toString(rowKey);
-    try {
-      Get getQueue = new Get(rowKey);
-      Result queue = getResultIfOwner(getQueue);
-      // The returned queue could be null if we have lost ownership of it
-      if (queue == null) {
-        abortable.abort(errMsg, new ReplicationException(errMsg));
-        return null;
-      }
-      return readWALsFromResult(queue);
-    } catch (IOException e) {
-      abortable.abort(errMsg, e);
-      return null;
-    }
-  }
-
-  @Override
-  public List<String> getAllQueues() {
-    List<String> allQueues = new ArrayList<String>();
-    ResultScanner queueScanner = null;
-    try {
-      queueScanner = this.getQueuesBelongingToServer(serverName);
-      for (Result queue : queueScanner) {
-        String rowKey =  Bytes.toString(queue.getRow());
-        // If the queue does not have a Owner History, then we must be its original owner. So we
-        // want to return its queueId in raw form
-        if (Bytes.toString(queue.getValue(CF, COL_OWNER_HISTORY)).length() == 0) {
-          allQueues.add(getRawQueueIdFromRowKey(rowKey));
-        } else {
-          allQueues.add(rowKey);
-        }
-      }
-      return allQueues;
-    } catch (IOException e) {
-      String errMsg = "Failed getting list of all replication queues";
-      abortable.abort(errMsg, e);
-      return null;
-    } finally {
-      if (queueScanner != null) {
-        queueScanner.close();
-      }
-    }
-  }
-
-  @Override
-  public Map<String, Set<String>> claimQueues(String regionserver) {
-    Map<String, Set<String>> queues = new HashMap<>();
-    if (isThisOurRegionServer(regionserver)) {
-      return queues;
-    }
-    ResultScanner queuesToClaim = null;
-    try {
-      queuesToClaim = this.getQueuesBelongingToServer(regionserver);
-      for (Result queue : queuesToClaim) {
-        if (attemptToClaimQueue(queue, regionserver)) {
-          String rowKey = Bytes.toString(queue.getRow());
-          ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(rowKey);
-          if (peerExists(replicationQueueInfo.getPeerId())) {
-            Set<String> sortedLogs = new HashSet<String>();
-            List<String> logs = getLogsInQueue(queue.getRow());
-            for (String log : logs) {
-              sortedLogs.add(log);
-            }
-            queues.put(rowKey, sortedLogs);
-            LOG.info(serverName + " has claimed queue " + rowKey + " from " + regionserver);
-          } else {
-            // Delete orphaned queues
-            removeQueue(Bytes.toString(queue.getRow()));
-            LOG.info(serverName + " has deleted abandoned queue " + rowKey + " from " +
-                regionserver);
-          }
-        }
-      }
-    } catch (IOException | KeeperException e) {
-      String errMsg = "Failed claiming queues for regionserver=" + regionserver;
-      abortable.abort(errMsg, e);
-      queues.clear();
-    } finally {
-      if (queuesToClaim != null) {
-        queuesToClaim.close();
-      }
-    }
-    return queues;
-  }
-
-  @Override
-  public List<String> getListOfReplicators() {
-    // scan all of the queues and return a list of all unique OWNER values
-    Set<String> peerServers = new HashSet<String>();
-    ResultScanner allQueuesInCluster = null;
-    try {
-      Scan scan = new Scan();
-      scan.addColumn(CF, COL_OWNER);
-      allQueuesInCluster = replicationTable.getScanner(scan);
-      for (Result queue : allQueuesInCluster) {
-        peerServers.add(Bytes.toString(queue.getValue(CF, COL_OWNER)));
-      }
-    } catch (IOException e) {
-      String errMsg = "Failed getting list of replicators";
-      abortable.abort(errMsg, e);
-    } finally {
-      if (allQueuesInCluster != null) {
-        allQueuesInCluster.close();
-      }
-    }
-    return new ArrayList<String>(peerServers);
-  }
-
-  @Override
-  public boolean isThisOurRegionServer(String regionserver) {
-    return this.serverName.equals(regionserver);
-  }
-
-  @Override
-  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
-    // TODO
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void removePeerFromHFileRefs(String peerId) {
-    // TODO
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void addHFileRefs(String peerId, List<String> files) throws ReplicationException {
-    // TODO
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void removeHFileRefs(String peerId, List<String> files) {
-    // TODO
-    throw new NotImplementedException();
-  }
-
-  /**
-   * Gets the Replication Table. Builds and blocks until the table is available if the Replication
-   * Table does not exist.
-   *
-   * @return the Replication Table
-   * @throws IOException if the Replication Table takes too long to build
-   */
-  private Table createAndGetReplicationTable() throws IOException {
-    if (!replicationTableExists()) {
-      createReplicationTable();
-    }
-    int maxRetries = conf.getInt("replication.queues.createtable.retries.number", 100);
-    RetryCounterFactory counterFactory = new RetryCounterFactory(maxRetries, 100);
-    RetryCounter retryCounter = counterFactory.create();
-    while (!replicationTableExists()) {
-      try {
-        retryCounter.sleepUntilNextRetry();
-        if (!retryCounter.shouldRetry()) {
-          throw new IOException("Unable to acquire the Replication Table");
-        }
-      } catch (InterruptedException e) {
-        return null;
-      }
-    }
-    return connection.getTable(REPLICATION_TABLE_NAME);
-  }
-
-  /**
-   * Checks whether the Replication Table exists yet
-   *
-   * @return whether the Replication Table exists
-   * @throws IOException
-   */
-  private boolean replicationTableExists() {
-    try {
-      return admin.tableExists(REPLICATION_TABLE_NAME);
-    } catch (IOException e) {
-      return false;
-    }
-  }
-
-  /**
-   * Create the replication table with the provided HColumnDescriptor REPLICATION_COL_DESCRIPTOR
-   * in ReplicationQueuesHBaseImpl
-   *
-   * @throws IOException
-   */
-  private void createReplicationTable() throws IOException {
-    HTableDescriptor replicationTableDescriptor = new HTableDescriptor(REPLICATION_TABLE_NAME);
-    replicationTableDescriptor.addFamily(REPLICATION_COL_DESCRIPTOR);
-    admin.createTable(replicationTableDescriptor);
-  }
-
-  /**
-   * Build the row key for the given queueId. This will uniquely identify it from all other queues
-   * in the cluster.
-   * @param serverName The owner of the queue
-   * @param queueId String identifier of the queue
-   * @return String representation of the queue's row key
-   */
-  private String buildQueueRowKey(String serverName, String queueId) {
-    return queueId + ROW_KEY_DELIMITER + serverName;
-  }
-
-  private String buildQueueRowKey(String queueId) {
-    return buildQueueRowKey(serverName, queueId);
-  }
-
-  /**
-   * Parse the original queueId from a row key
-   * @param rowKey String representation of a queue's row key
-   * @return the original queueId
-   */
-  private String getRawQueueIdFromRowKey(String rowKey) {
-    return rowKey.split(ROW_KEY_DELIMITER)[0];
-  }
-
-  /**
-   * See safeQueueUpdate(RowMutations mutate)
-   *
-   * @param put Row mutation to perform on the queue
-   */
-  private void safeQueueUpdate(Put put) throws ReplicationException, IOException {
-    RowMutations mutations = new RowMutations(put.getRow());
-    mutations.add(put);
-    safeQueueUpdate(mutations);
-  }
-
-  /**
-   * See safeQueueUpdate(RowMutations mutate)
-   *
-   * @param delete Row mutation to perform on the queue
-   */
-  private void safeQueueUpdate(Delete delete) throws ReplicationException,
-      IOException{
-    RowMutations mutations = new RowMutations(delete.getRow());
-    mutations.add(delete);
-    safeQueueUpdate(mutations);
-  }
-
-  /**
-   * Attempt to mutate a given queue in the Replication Table with a checkAndPut on the OWNER column
-   * of the queue. Abort the server if this checkAndPut fails: which means we have somehow lost
-   * ownership of the column or an IO Exception has occurred during the transaction.
-   *
-   * @param mutate Mutation to perform on a given queue
-   */
-  private void safeQueueUpdate(RowMutations mutate) throws ReplicationException, IOException{
-    boolean updateSuccess = replicationTable.checkAndMutate(mutate.getRow(), CF, COL_OWNER,
-      CompareFilter.CompareOp.EQUAL, serverNameBytes, mutate);
-    if (!updateSuccess) {
-      throw new ReplicationException("Failed to update Replication Table because we lost queue " +
-        " ownership");
-    }
-  }
-
-  /**
-   * Returns a queue's row key given either its raw or reclaimed queueId
-   *
-   * @param queueId queueId of the queue
-   * @return byte representation of the queue's row key
-   */
-  private byte[] queueIdToRowKey(String queueId) {
-    // Cluster id's are guaranteed to have no hyphens, so if the passed in queueId has no hyphen
-    // then this is not a reclaimed queue.
-    if (!queueId.contains(ROW_KEY_DELIMITER)) {
-      return Bytes.toBytes(buildQueueRowKey(queueId));
-      // If the queueId contained some hyphen it was reclaimed. In this case, the queueId is the
-      // queue's row key
-    } else {
-      return Bytes.toBytes(queueId);
-    }
-  }
-
-  /**
-   * Get the QueueIds belonging to the named server from the ReplicationTable
-   *
-   * @param server name of the server
-   * @return a ResultScanner over the QueueIds belonging to the server
-   * @throws IOException
-   */
-  private ResultScanner getQueuesBelongingToServer(String server) throws IOException {
-    Scan scan = new Scan();
-    SingleColumnValueFilter filterMyQueues = new SingleColumnValueFilter(CF, COL_OWNER,
-      CompareFilter.CompareOp.EQUAL, Bytes.toBytes(server));
-    scan.setFilter(filterMyQueues);
-    scan.addColumn(CF, COL_OWNER);
-    scan.addColumn(CF, COL_OWNER_HISTORY);
-    ResultScanner results = replicationTable.getScanner(scan);
-    return results;
-  }
-
-  /**
-   * Check if the queue specified by queueId is stored in HBase
-   *
-   * @param queueId Either raw or reclaimed format of the queueId
-   * @return Whether the queue is stored in HBase
-   * @throws IOException
-   */
-  private boolean checkQueueExists(String queueId) throws IOException {
-    byte[] rowKey = queueIdToRowKey(queueId);
-    return replicationTable.exists(new Get(rowKey));
-  }
-
-  /**
-   * Read all of the WAL's from a queue into a list
-   *
-   * @param queue HBase query result containing the queue
-   * @return a list of all the WAL filenames
-   */
-  private List<String> readWALsFromResult(Result queue) {
-    List<String> wals = new ArrayList<>();
-    Map<byte[], byte[]> familyMap = queue.getFamilyMap(CF);
-    for(byte[] cQualifier : familyMap.keySet()) {
-      // Ignore the meta data fields of the queue
-      if (Arrays.equals(cQualifier, COL_OWNER) || Arrays.equals(cQualifier, COL_OWNER_HISTORY)) {
-        continue;
-      }
-      wals.add(Bytes.toString(cQualifier));
-    }
-    return wals;
-  }
-
-  /**
-   * Attempt to claim the given queue with a checkAndPut on the OWNER column. We check that the
-   * recently killed server is still the OWNER before we claim it.
-   *
-   * @param queue The queue that we are trying to claim
-   * @param originalServer The server that originally owned the queue
-   * @return Whether we successfully claimed the queue
-   * @throws IOException
-   */
-  private boolean attemptToClaimQueue (Result queue, String originalServer) throws IOException{
-    Put putQueueNameAndHistory = new Put(queue.getRow());
-    putQueueNameAndHistory.addColumn(CF, COL_OWNER, Bytes.toBytes(serverName));
-    String newOwnerHistory = buildClaimedQueueHistory(Bytes.toString(queue.getValue(CF,
-      COL_OWNER_HISTORY)), originalServer);
-    putQueueNameAndHistory.addColumn(CF, COL_OWNER_HISTORY, Bytes.toBytes(newOwnerHistory));
-    RowMutations claimAndRenameQueue = new RowMutations(queue.getRow());
-    claimAndRenameQueue.add(putQueueNameAndHistory);
-    // Attempt to claim ownership for this queue by checking if the current OWNER is the original
-    // server. If it is not then another RS has already claimed it. If it is we set ourselves as the
-    // new owner and update the queue's history
-    boolean success = replicationTable.checkAndMutate(queue.getRow(), CF, COL_OWNER,
-      CompareFilter.CompareOp.EQUAL, Bytes.toBytes(originalServer), claimAndRenameQueue);
-    return success;
-  }
-
-  /**
-   * Creates a "|" delimited record of the queue's past region server owners.
-   *
-   * @param originalHistory the queue's original owner history
-   * @param oldServer the name of the server that used to own the queue
-   * @return the queue's new owner history
-   */
-  private String buildClaimedQueueHistory(String originalHistory, String oldServer) {
-    return originalHistory + "|" + oldServer;
-  }
-
-  /**
-   * Attempts to run a Get on some queue. Will only return a non-null result if we currently own
-   * the queue.
-   *
-   * @param get The get that we want to query
-   * @return The result of the get if this server is the owner of the queue. Else it returns null
-   * @throws IOException
-   */
-  private Result getResultIfOwner(Get get) throws IOException {
-    Scan scan = new Scan(get);
-    // Check if the Get currently contains all columns or only specific columns
-    if (scan.getFamilyMap().size() > 0) {
-      // Add the OWNER column if the scan is already only over specific columns
-      scan.addColumn(CF, COL_OWNER);
-    }
-    scan.setMaxResultSize(1);
-    SingleColumnValueFilter checkOwner = new SingleColumnValueFilter(CF, COL_OWNER,
-      CompareFilter.CompareOp.EQUAL, serverNameBytes);
-    scan.setFilter(checkOwner);
-    ResultScanner scanner = null;
-    try {
-      scanner = replicationTable.getScanner(scan);
-      Result result = scanner.next();
-      return (result == null || result.isEmpty()) ? null : result;
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
new file mode 100644
index 0000000..c1506cd
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
@@ -0,0 +1,351 @@
+/*
+*
+* 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/*
+ * Abstract class that provides an interface to the Replication Table. Which is currently
+ * being used for WAL offset tracking.
+ * The basic schema of this table will store each individual queue as a
+ * seperate row. The row key will be a unique identifier of the creating server's name and the
+ * queueId. Each queue must have the following two columns:
+ *  COL_QUEUE_OWNER: tracks which server is currently responsible for tracking the queue
+ *  COL_QUEUE_OWNER_HISTORY: a "|" delimited list of the previous server's that have owned this
+ *    queue. The most recent previous owner is the leftmost entry.
+ * They will also have columns mapping [WAL filename : offset]
+ */
+
+@InterfaceAudience.Private
+abstract class ReplicationTableBase {
+
+  /** Name of the HBase Table used for tracking replication*/
+  public static final TableName REPLICATION_TABLE_NAME =
+    TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication");
+
+  // Column family and column names for Queues in the Replication Table
+  public static final byte[] CF_QUEUE = Bytes.toBytes("q");
+  public static final byte[] COL_QUEUE_OWNER = Bytes.toBytes("o");
+  public static final byte[] COL_QUEUE_OWNER_HISTORY = Bytes.toBytes("h");
+
+  // Column Descriptor for the Replication Table
+  private static final HColumnDescriptor REPLICATION_COL_DESCRIPTOR =
+    new HColumnDescriptor(CF_QUEUE).setMaxVersions(1)
+      .setInMemory(true)
+      .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+        // TODO: Figure out which bloom filter to use
+      .setBloomFilterType(BloomType.NONE);
+
+  // The value used to delimit the queueId and server name inside of a queue's row key. Currently a
+  // hyphen, because it is guaranteed that queueId (which is a cluster id) cannot contain hyphens.
+  // See HBASE-11394.
+  public static final String ROW_KEY_DELIMITER = "-";
+
+  // The value used to delimit server names in the queue history list
+  public static final String QUEUE_HISTORY_DELIMITER = "|";
+
+  /*
+  * Make sure that HBase table operations for replication have a high number of retries. This is
+  * because the server is aborted if any HBase table operation fails. Each RPC will be attempted
+  * 3600 times before exiting. This provides each operation with 2 hours of retries
+  * before the server is aborted.
+  */
+  private static final int CLIENT_RETRIES = 3600;
+  private static final int RPC_TIMEOUT = 2000;
+  private static final int OPERATION_TIMEOUT = CLIENT_RETRIES * RPC_TIMEOUT;
+
+  protected final Table replicationTable;
+  protected final Configuration conf;
+  protected final Abortable abortable;
+  private final Admin admin;
+  private final Connection connection;
+
+  public ReplicationTableBase(Configuration conf, Abortable abort) throws IOException {
+    this.conf = new Configuration(conf);
+    this.abortable = abort;
+    decorateConf();
+    this.connection = ConnectionFactory.createConnection(this.conf);
+    this.admin = connection.getAdmin();
+    this.replicationTable = createAndGetReplicationTable();
+    setTableTimeOuts();
+  }
+
+  /**
+   * Modify the connection's config so that operations run on the Replication Table have longer and
+   * a larger number of retries
+   */
+  private void decorateConf() {
+    this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES);
+  }
+
+  /**
+   * Increases the RPC and operations timeouts for the Replication Table
+   */
+  private void setTableTimeOuts() {
+    replicationTable.setRpcTimeout(RPC_TIMEOUT);
+    replicationTable.setOperationTimeout(OPERATION_TIMEOUT);
+  }
+
+  /**
+   * Build the row key for the given queueId. This will uniquely identify it from all other queues
+   * in the cluster.
+   * @param serverName The owner of the queue
+   * @param queueId String identifier of the queue
+   * @return String representation of the queue's row key
+   */
+  protected String buildQueueRowKey(String serverName, String queueId) {
+    return queueId + ROW_KEY_DELIMITER + serverName;
+  }
+
+  /**
+   * Parse the original queueId from a row key
+   * @param rowKey String representation of a queue's row key
+   * @return the original queueId
+   */
+  protected String getRawQueueIdFromRowKey(String rowKey) {
+    return rowKey.split(ROW_KEY_DELIMITER)[0];
+  }
+
+  /**
+   * Returns a queue's row key given either its raw or reclaimed queueId
+   *
+   * @param queueId queueId of the queue
+   * @return byte representation of the queue's row key
+   */
+  protected byte[] queueIdToRowKey(String serverName, String queueId) {
+    // Cluster id's are guaranteed to have no hyphens, so if the passed in queueId has no hyphen
+    // then this is not a reclaimed queue.
+    if (!queueId.contains(ROW_KEY_DELIMITER)) {
+      return Bytes.toBytes(buildQueueRowKey(serverName, queueId));
+      // If the queueId contained some hyphen it was reclaimed. In this case, the queueId is the
+      // queue's row key
+    } else {
+      return Bytes.toBytes(queueId);
+    }
+  }
+
+  /**
+   * Creates a "|" delimited record of the queue's past region server owners.
+   *
+   * @param originalHistory the queue's original owner history
+   * @param oldServer the name of the server that used to own the queue
+   * @return the queue's new owner history
+   */
+  protected String buildClaimedQueueHistory(String originalHistory, String oldServer) {
+    return oldServer + QUEUE_HISTORY_DELIMITER + originalHistory;
+  }
+
+  /**
+   * Get a list of all region servers that have outstanding replication queues. These servers could
+   * be alive, dead or from a previous run of the cluster.
+   * @return a list of server names
+   */
+  protected List<String> getListOfReplicators() {
+    // scan all of the queues and return a list of all unique OWNER values
+    Set<String> peerServers = new HashSet<String>();
+    ResultScanner allQueuesInCluster = null;
+    try {
+      Scan scan = new Scan();
+      scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
+      allQueuesInCluster = replicationTable.getScanner(scan);
+      for (Result queue : allQueuesInCluster) {
+        peerServers.add(Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER)));
+      }
+    } catch (IOException e) {
+      String errMsg = "Failed getting list of replicators";
+      abortable.abort(errMsg, e);
+    } finally {
+      if (allQueuesInCluster != null) {
+        allQueuesInCluster.close();
+      }
+    }
+    return new ArrayList<String>(peerServers);
+  }
+
+  protected List<String> getAllQueues(String serverName) {
+    List<String> allQueues = new ArrayList<String>();
+    ResultScanner queueScanner = null;
+    try {
+      queueScanner = getQueuesBelongingToServer(serverName);
+      for (Result queue : queueScanner) {
+        String rowKey =  Bytes.toString(queue.getRow());
+        // If the queue does not have a Owner History, then we must be its original owner. So we
+        // want to return its queueId in raw form
+        if (Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER_HISTORY)).length() == 0) {
+          allQueues.add(getRawQueueIdFromRowKey(rowKey));
+        } else {
+          allQueues.add(rowKey);
+        }
+      }
+      return allQueues;
+    } catch (IOException e) {
+      String errMsg = "Failed getting list of all replication queues for serverName=" + serverName;
+      abortable.abort(errMsg, e);
+      return null;
+    } finally {
+      if (queueScanner != null) {
+        queueScanner.close();
+      }
+    }
+  }
+
+  protected List<String> getLogsInQueue(String serverName, String queueId) {
+    String rowKey = queueId;
+    if (!queueId.contains(ROW_KEY_DELIMITER)) {
+      rowKey = buildQueueRowKey(serverName, queueId);
+    }
+    return getLogsInQueue(Bytes.toBytes(rowKey));
+  }
+
+  protected List<String> getLogsInQueue(byte[] rowKey) {
+    String errMsg = "Failed getting logs in queue queueId=" + Bytes.toString(rowKey);
+    try {
+      Get getQueue = new Get(rowKey);
+      Result queue = replicationTable.get(getQueue);
+      if (queue == null || queue.isEmpty()) {
+        abortable.abort(errMsg, new ReplicationException(errMsg));
+        return null;
+      }
+      return readWALsFromResult(queue);
+    } catch (IOException e) {
+      abortable.abort(errMsg, e);
+      return null;
+    }
+  }
+
+  /**
+   * Read all of the WAL's from a queue into a list
+   *
+   * @param queue HBase query result containing the queue
+   * @return a list of all the WAL filenames
+   */
+  protected List<String> readWALsFromResult(Result queue) {
+    List<String> wals = new ArrayList<>();
+    Map<byte[], byte[]> familyMap = queue.getFamilyMap(CF_QUEUE);
+    for (byte[] cQualifier : familyMap.keySet()) {
+      // Ignore the meta data fields of the queue
+      if (Arrays.equals(cQualifier, COL_QUEUE_OWNER) || Arrays.equals(cQualifier,
+          COL_QUEUE_OWNER_HISTORY)) {
+        continue;
+      }
+      wals.add(Bytes.toString(cQualifier));
+    }
+    return wals;
+  }
+
+  /**
+   * Get the queue id's and meta data (Owner and History) for the queues belonging to the named
+   * server
+   *
+   * @param server name of the server
+   * @return a ResultScanner over the QueueIds belonging to the server
+   * @throws IOException
+   */
+  private ResultScanner getQueuesBelongingToServer(String server) throws IOException {
+    Scan scan = new Scan();
+    SingleColumnValueFilter filterMyQueues = new SingleColumnValueFilter(CF_QUEUE, COL_QUEUE_OWNER,
+      CompareFilter.CompareOp.EQUAL, Bytes.toBytes(server));
+    scan.setFilter(filterMyQueues);
+    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
+    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY);
+    ResultScanner results = replicationTable.getScanner(scan);
+    return results;
+  }
+
+  /**
+   * Gets the Replication Table. Builds and blocks until the table is available if the Replication
+   * Table does not exist.
+   *
+   * @return the Replication Table
+   * @throws IOException if the Replication Table takes too long to build
+   */
+  private Table createAndGetReplicationTable() throws IOException {
+    if (!replicationTableExists()) {
+      createReplicationTable();
+    }
+    int maxRetries = conf.getInt("replication.queues.createtable.retries.number", 100);
+    RetryCounterFactory counterFactory = new RetryCounterFactory(maxRetries, 100);
+    RetryCounter retryCounter = counterFactory.create();
+    while (!replicationTableExists()) {
+      try {
+        retryCounter.sleepUntilNextRetry();
+        if (!retryCounter.shouldRetry()) {
+          throw new IOException("Unable to acquire the Replication Table");
+        }
+      } catch (InterruptedException e) {
+        return null;
+      }
+    }
+    return connection.getTable(REPLICATION_TABLE_NAME);
+  }
+
+  /**
+   * Create the replication table with the provided HColumnDescriptor REPLICATION_COL_DESCRIPTOR
+   * in TableBasedReplicationQueuesImpl
+   * @throws IOException
+   */
+  private void createReplicationTable() throws IOException {
+    HTableDescriptor replicationTableDescriptor = new HTableDescriptor(REPLICATION_TABLE_NAME);
+    replicationTableDescriptor.addFamily(REPLICATION_COL_DESCRIPTOR);
+    admin.createTable(replicationTableDescriptor);
+  }
+
+  /**
+   * Checks whether the Replication Table exists yet
+   *
+   * @return whether the Replication Table exists
+   * @throws IOException
+   */
+  private boolean replicationTableExists() {
+    try {
+      return admin.tableExists(REPLICATION_TABLE_NAME);
+    } catch (IOException e) {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java
new file mode 100644
index 0000000..55dfdd8
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesClientImpl.java
@@ -0,0 +1,111 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Implements the ReplicationQueuesClient interface on top of the Replication Table. It utilizes
+ * the ReplicationTableBase to access the Replication Table.
+ */
+@InterfaceAudience.Private
+public class TableBasedReplicationQueuesClientImpl extends ReplicationTableBase
+  implements ReplicationQueuesClient {
+
+  public TableBasedReplicationQueuesClientImpl(ReplicationQueuesClientArguments args)
+    throws IOException {
+    super(args.getConf(), args.getAbortable());
+  }
+  public TableBasedReplicationQueuesClientImpl(Configuration conf,
+                                               Abortable abortable) throws IOException {
+    super(conf, abortable);
+  }
+
+  @Override
+  public void init() throws ReplicationException{
+    // no-op
+  }
+
+  @Override
+  public List<String> getListOfReplicators() {
+    return super.getListOfReplicators();
+  }
+
+  @Override
+  public List<String> getLogsInQueue(String serverName, String queueId) {
+    return super.getLogsInQueue(serverName, queueId);
+  }
+
+  @Override
+  public List<String> getAllQueues(String serverName) {
+    return super.getAllQueues(serverName);
+  }
+
+  @Override
+  public Set<String> getAllWALs() {
+    Set<String> allWals = new HashSet<String>();
+    ResultScanner allQueues = null;
+    try {
+      allQueues = replicationTable.getScanner(new Scan());
+      for (Result queue : allQueues) {
+        for (String wal : readWALsFromResult(queue)) {
+          allWals.add(wal);
+        }
+      }
+    } catch (IOException e) {
+      String errMsg = "Failed getting all WAL's in Replication Table";
+      abortable.abort(errMsg, e);
+    } finally {
+      if (allQueues != null) {
+        allQueues.close();
+      }
+    }
+    return allWals;
+  }
+
+  @Override
+  public int getHFileRefsNodeChangeVersion() throws KeeperException {
+    // TODO
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
+    // TODO
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public List<String> getReplicableHFiles(String peerId) throws KeeperException {
+    // TODO
+    throw new NotImplementedException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
new file mode 100644
index 0000000..6ea7801
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
@@ -0,0 +1,437 @@
+/*
+*
+* 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;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class provides an implementation of the ReplicationQueues interface using an HBase table
+ * "Replication Table". It utilizes the ReplicationTableBase to access the Replication Table.
+ */
+@InterfaceAudience.Private
+public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
+  implements ReplicationQueues {
+
+  private static final Log LOG = LogFactory.getLog(TableBasedReplicationQueuesImpl.class);
+
+  // Common byte values used in replication offset tracking
+  private static final byte[] INITIAL_OFFSET_BYTES = Bytes.toBytes(0L);
+  private static final byte[] EMPTY_STRING_BYTES = Bytes.toBytes("");
+
+  private String serverName = null;
+  private byte[] serverNameBytes = null;
+
+  // TODO: Only use this variable temporarily. Eventually we want to use HBase to store all
+  // TODO: replication information
+  private ReplicationStateZKBase replicationState;
+
+  public TableBasedReplicationQueuesImpl(ReplicationQueuesArguments args) throws IOException {
+    this(args.getConf(), args.getAbortable(), args.getZk());
+  }
+
+  public TableBasedReplicationQueuesImpl(Configuration conf, Abortable abort, ZooKeeperWatcher zkw)
+    throws IOException {
+    super(conf, abort);
+    replicationState = new ReplicationStateZKBase(zkw, conf, abort) {};
+  }
+
+  @Override
+  public void init(String serverName) throws ReplicationException {
+    this.serverName = serverName;
+    this.serverNameBytes = Bytes.toBytes(serverName);
+  }
+
+  @Override
+  public List<String> getListOfReplicators() {
+    return super.getListOfReplicators();
+  }
+
+  @Override
+  public void removeQueue(String queueId) {
+
+    try {
+      byte[] rowKey = queueIdToRowKey(queueId);
+      Delete deleteQueue = new Delete(rowKey);
+      safeQueueUpdate(deleteQueue);
+    } catch (IOException | ReplicationException e) {
+      String errMsg = "Failed removing queue queueId=" + queueId;
+      abortable.abort(errMsg, e);
+    }
+  }
+
+  @Override
+  public void addLog(String queueId, String filename) throws ReplicationException {
+    try {
+      if (!checkQueueExists(queueId)) {
+        // Each queue will have an Owner, OwnerHistory, and a collection of [WAL:offset] key values
+        Put putNewQueue = new Put(Bytes.toBytes(buildQueueRowKey(queueId)));
+        putNewQueue.addColumn(CF_QUEUE, COL_QUEUE_OWNER, serverNameBytes);
+        putNewQueue.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY, EMPTY_STRING_BYTES);
+        putNewQueue.addColumn(CF_QUEUE, Bytes.toBytes(filename), INITIAL_OFFSET_BYTES);
+        replicationTable.put(putNewQueue);
+      } else {
+        // Otherwise simply add the new log and offset as a new column
+        Put putNewLog = new Put(queueIdToRowKey(queueId));
+        putNewLog.addColumn(CF_QUEUE, Bytes.toBytes(filename), INITIAL_OFFSET_BYTES);
+        safeQueueUpdate(putNewLog);
+      }
+    } catch (IOException | ReplicationException e) {
+      String errMsg = "Failed adding log queueId=" + queueId + " filename=" + filename;
+      abortable.abort(errMsg, e);
+    }
+  }
+
+  @Override
+  public void removeLog(String queueId, String filename) {
+    try {
+      byte[] rowKey = queueIdToRowKey(queueId);
+      Delete delete = new Delete(rowKey);
+      delete.addColumns(CF_QUEUE, Bytes.toBytes(filename));
+      safeQueueUpdate(delete);
+    } catch (IOException | ReplicationException e) {
+      String errMsg = "Failed removing log queueId=" + queueId + " filename=" + filename;
+      abortable.abort(errMsg, e);
+    }
+  }
+
+  @Override
+  public void setLogPosition(String queueId, String filename, long position) {
+    try {
+      byte[] rowKey = queueIdToRowKey(queueId);
+      // Check that the log exists. addLog() must have been called before setLogPosition().
+      Get checkLogExists = new Get(rowKey);
+      checkLogExists.addColumn(CF_QUEUE, Bytes.toBytes(filename));
+      if (!replicationTable.exists(checkLogExists)) {
+        String errMsg = "Could not set position of non-existent log from queueId=" + queueId +
+          ", filename=" + filename;
+        abortable.abort(errMsg, new ReplicationException(errMsg));
+        return;
+      }
+      // Update the log offset if it exists
+      Put walAndOffset = new Put(rowKey);
+      walAndOffset.addColumn(CF_QUEUE, Bytes.toBytes(filename), Bytes.toBytes(position));
+      safeQueueUpdate(walAndOffset);
+    } catch (IOException | ReplicationException e) {
+      String errMsg = "Failed writing log position queueId=" + queueId + "filename=" +
+        filename + " position=" + position;
+      abortable.abort(errMsg, e);
+    }
+  }
+
+  @Override
+  public long getLogPosition(String queueId, String filename) throws ReplicationException {
+    try {
+      byte[] rowKey = queueIdToRowKey(queueId);
+      Get getOffset = new Get(rowKey);
+      getOffset.addColumn(CF_QUEUE, Bytes.toBytes(filename));
+      Result result = getResultIfOwner(getOffset);
+      if (result == null || !result.containsColumn(CF_QUEUE, Bytes.toBytes(filename))) {
+        throw new ReplicationException("Could not read empty result while getting log position " +
+          "queueId=" + queueId + ", filename=" + filename);
+      }
+      return Bytes.toLong(result.getValue(CF_QUEUE, Bytes.toBytes(filename)));
+    } catch (IOException e) {
+      throw new ReplicationException("Could not get position in log for queueId=" + queueId +
+        ", filename=" + filename);
+    }
+  }
+
+  @Override
+  public void removeAllQueues() {
+    List<String> myQueueIds = getAllQueues();
+    for (String queueId : myQueueIds) {
+      removeQueue(queueId);
+    }
+  }
+
+  @Override
+  public List<String> getLogsInQueue(String queueId) {
+    byte[] rowKey = queueIdToRowKey(queueId);
+    return getLogsInQueueAndCheckOwnership(rowKey);
+  }
+
+  @Override
+  public List<String> getAllQueues() {
+    return getAllQueues(serverName);
+  }
+
+  @Override
+  public Map<String, Set<String>> claimQueues(String regionserver) {
+    Map<String, Set<String>> queues = new HashMap<>();
+    if (isThisOurRegionServer(regionserver)) {
+      return queues;
+    }
+    ResultScanner queuesToClaim = null;
+    try {
+      queuesToClaim = getAllQueuesScanner(regionserver);
+      for (Result queue : queuesToClaim) {
+        if (attemptToClaimQueue(queue, regionserver)) {
+          String rowKey = Bytes.toString(queue.getRow());
+          ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(rowKey);
+          if (replicationState.peerExists(replicationQueueInfo.getPeerId())) {
+            Set<String> sortedLogs = new HashSet<String>();
+            List<String> logs = getLogsInQueue(queue.getRow());
+            for (String log : logs) {
+              sortedLogs.add(log);
+            }
+            queues.put(rowKey, sortedLogs);
+            LOG.info(serverName + " has claimed queue " + rowKey + " from " + regionserver);
+          } else {
+            // Delete orphaned queues
+            removeQueue(Bytes.toString(queue.getRow()));
+            LOG.info(serverName + " has deleted abandoned queue " + rowKey + " from " +
+              regionserver);
+          }
+        }
+      }
+    } catch (IOException | KeeperException e) {
+      String errMsg = "Failed claiming queues for regionserver=" + regionserver;
+      abortable.abort(errMsg, e);
+      queues.clear();
+    } finally {
+      if (queuesToClaim != null) {
+        queuesToClaim.close();
+      }
+    }
+    return queues;
+  }
+
+  /**
+   * Get the QueueIds belonging to the named server from the ReplicationTableBase
+   *
+   * @param server name of the server
+   * @return a ResultScanner over the QueueIds belonging to the server
+   * @throws IOException
+   */
+  private ResultScanner getAllQueuesScanner(String server) throws IOException {
+    Scan scan = new Scan();
+    SingleColumnValueFilter filterMyQueues = new SingleColumnValueFilter(CF_QUEUE, COL_QUEUE_OWNER,
+      CompareFilter.CompareOp.EQUAL, Bytes.toBytes(server));
+    scan.setFilter(filterMyQueues);
+    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
+    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY);
+    ResultScanner results = replicationTable.getScanner(scan);
+    return results;
+  }
+
+  @Override
+  public boolean isThisOurRegionServer(String regionserver) {
+    return this.serverName.equals(regionserver);
+  }
+
+  @Override
+  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
+    // TODO
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void removePeerFromHFileRefs(String peerId) {
+    // TODO
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void addHFileRefs(String peerId, List<String> files) throws ReplicationException {
+    // TODO
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void removeHFileRefs(String peerId, List<String> files) {
+    // TODO
+    throw new NotImplementedException();
+  }
+
+  private List<String> getLogsInQueueAndCheckOwnership(byte[] rowKey) {
+    String errMsg = "Failed getting logs in queue queueId=" + Bytes.toString(rowKey);
+    List<String> logs = new ArrayList<String>();
+    try {
+      Get getQueue = new Get(rowKey);
+      Result queue = getResultIfOwner(getQueue);
+      if (queue == null || queue.isEmpty()) {
+        String errMsgLostOwnership = "Failed getting logs for queue queueId=" +
+          Bytes.toString(rowKey) + " because the queue was missing or we lost ownership";
+        abortable.abort(errMsg, new ReplicationException(errMsgLostOwnership));
+        return null;
+      }
+      Map<byte[], byte[]> familyMap = queue.getFamilyMap(CF_QUEUE);
+      for(byte[] cQualifier : familyMap.keySet()) {
+        if (Arrays.equals(cQualifier, COL_QUEUE_OWNER) || Arrays.equals(cQualifier,
+            COL_QUEUE_OWNER_HISTORY)) {
+          continue;
+        }
+        logs.add(Bytes.toString(cQualifier));
+      }
+    } catch (IOException e) {
+      abortable.abort(errMsg, e);
+      return null;
+    }
+    return logs;
+  }
+
+  private String buildQueueRowKey(String queueId) {
+    return buildQueueRowKey(serverName, queueId);
+  }
+
+  /**
+   * Convenience method that gets the row key of the queue specified by queueId
+   * @param queueId queueId of a queue in this server
+   * @return the row key of the queue in the Replication Table
+   */
+  private byte[] queueIdToRowKey(String queueId) {
+    return queueIdToRowKey(serverName, queueId);
+  }
+
+  /**
+   * See safeQueueUpdate(RowMutations mutate)
+   *
+   * @param put Row mutation to perform on the queue
+   */
+  private void safeQueueUpdate(Put put) throws ReplicationException, IOException {
+    RowMutations mutations = new RowMutations(put.getRow());
+    mutations.add(put);
+    safeQueueUpdate(mutations);
+  }
+
+  /**
+   * See safeQueueUpdate(RowMutations mutate)
+   *
+   * @param delete Row mutation to perform on the queue
+   */
+  private void safeQueueUpdate(Delete delete) throws ReplicationException,
+    IOException{
+    RowMutations mutations = new RowMutations(delete.getRow());
+    mutations.add(delete);
+    safeQueueUpdate(mutations);
+  }
+
+  /**
+   * Attempt to mutate a given queue in the Replication Table with a checkAndPut on the OWNER column
+   * of the queue. Abort the server if this checkAndPut fails: which means we have somehow lost
+   * ownership of the column or an IO Exception has occurred during the transaction.
+   *
+   * @param mutate Mutation to perform on a given queue
+   */
+  private void safeQueueUpdate(RowMutations mutate) throws ReplicationException, IOException{
+    boolean updateSuccess = replicationTable.checkAndMutate(mutate.getRow(), CF_QUEUE,
+        COL_QUEUE_OWNER, CompareFilter.CompareOp.EQUAL, serverNameBytes, mutate);
+    if (!updateSuccess) {
+      throw new ReplicationException("Failed to update Replication Table because we lost queue " +
+        " ownership");
+    }
+  }
+
+  /**
+   * Check if the queue specified by queueId is stored in HBase
+   *
+   * @param queueId Either raw or reclaimed format of the queueId
+   * @return Whether the queue is stored in HBase
+   * @throws IOException
+   */
+  private boolean checkQueueExists(String queueId) throws IOException {
+    byte[] rowKey = queueIdToRowKey(queueId);
+    return replicationTable.exists(new Get(rowKey));
+  }
+
+  /**
+   * Attempt to claim the given queue with a checkAndPut on the OWNER column. We check that the
+   * recently killed server is still the OWNER before we claim it.
+   *
+   * @param queue The queue that we are trying to claim
+   * @param originalServer The server that originally owned the queue
+   * @return Whether we successfully claimed the queue
+   * @throws IOException
+   */
+  private boolean attemptToClaimQueue (Result queue, String originalServer) throws IOException{
+    Put putQueueNameAndHistory = new Put(queue.getRow());
+    putQueueNameAndHistory.addColumn(CF_QUEUE, COL_QUEUE_OWNER, Bytes.toBytes(serverName));
+    String newOwnerHistory = buildClaimedQueueHistory(Bytes.toString(queue.getValue(CF_QUEUE,
+      COL_QUEUE_OWNER_HISTORY)), originalServer);
+    putQueueNameAndHistory.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY,
+        Bytes.toBytes(newOwnerHistory));
+    RowMutations claimAndRenameQueue = new RowMutations(queue.getRow());
+    claimAndRenameQueue.add(putQueueNameAndHistory);
+    // Attempt to claim ownership for this queue by checking if the current OWNER is the original
+    // server. If it is not then another RS has already claimed it. If it is we set ourselves as the
+    // new owner and update the queue's history
+    boolean success = replicationTable.checkAndMutate(queue.getRow(), CF_QUEUE, COL_QUEUE_OWNER,
+      CompareFilter.CompareOp.EQUAL, Bytes.toBytes(originalServer), claimAndRenameQueue);
+    return success;
+  }
+
+  /**
+   * Attempts to run a Get on some queue. Will only return a non-null result if we currently own
+   * the queue.
+   *
+   * @param get The Get that we want to query
+   * @return The result of the Get if this server is the owner of the queue. Else it returns null.
+   * @throws IOException
+   */
+  private Result getResultIfOwner(Get get) throws IOException {
+    Scan scan = new Scan(get);
+    // Check if the Get currently contains all columns or only specific columns
+    if (scan.getFamilyMap().size() > 0) {
+      // Add the OWNER column if the scan is already only over specific columns
+      scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
+    }
+    scan.setMaxResultSize(1);
+    SingleColumnValueFilter checkOwner = new SingleColumnValueFilter(CF_QUEUE, COL_QUEUE_OWNER,
+      CompareFilter.CompareOp.EQUAL, serverNameBytes);
+    scan.setFilter(checkOwner);
+    ResultScanner scanner = null;
+    try {
+      scanner = replicationTable.getScanner(scan);
+      Result result = scanner.next();
+      return (result == null || result.isEmpty()) ? null : result;
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
index 5df9379..a7b2f26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
@@ -141,15 +142,16 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     super.setConf(conf);
     try {
       initReplicationQueuesClient(conf, zk);
-    } catch (IOException e) {
+    } catch (Exception e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
 
   private void initReplicationQueuesClient(Configuration conf, ZooKeeperWatcher zk)
-      throws ZooKeeperConnectionException, IOException {
+      throws Exception {
     this.zkw = zk;
-    this.rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, new WarnOnlyAbortable());
+    this.rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(
+        conf, new WarnOnlyAbortable(), zkw));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 9ecba11..9e724db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import java.io.IOException;
 import java.util.Collections;
@@ -67,7 +68,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     try {
       // The concurrently created new WALs may not be included in the return list,
       // but they won't be deleted because they're not in the checking set.
-      wals = loadWALsFromQueues();
+      wals = replicationQueues.getAllWALs();
     } catch (KeeperException e) {
       LOG.warn("Failed to read zookeeper, skipping checking deletable files");
       return Collections.emptyList();
@@ -88,43 +89,6 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
       }});
   }
 
-  /**
-   * Load all wals in all replication queues from ZK. This method guarantees to return a
-   * snapshot which contains all WALs in the zookeeper at the start of this call even there
-   * is concurrent queue failover. However, some newly created WALs during the call may
-   * not be included.
-   */
-  private Set<String> loadWALsFromQueues() throws KeeperException {
-    for (int retry = 0; ; retry++) {
-      int v0 = replicationQueues.getQueuesZNodeCversion();
-      List<String> rss = replicationQueues.getListOfReplicators();
-      if (rss == null) {
-        LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
-        return ImmutableSet.of();
-      }
-      Set<String> wals = Sets.newHashSet();
-      for (String rs : rss) {
-        List<String> listOfPeers = replicationQueues.getAllQueues(rs);
-        // if rs just died, this will be null
-        if (listOfPeers == null) {
-          continue;
-        }
-        for (String id : listOfPeers) {
-          List<String> peersWals = replicationQueues.getLogsInQueue(rs, id);
-          if (peersWals != null) {
-            wals.addAll(peersWals);
-          }
-        }
-      }
-      int v1 = replicationQueues.getQueuesZNodeCversion();
-      if (v0 == v1) {
-        return wals;
-      }
-      LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
-          v0, v1, retry));
-    }
-  }
-
   @Override
   public void setConf(Configuration config) {
     // If replication is disabled, keep all members null
@@ -148,10 +112,10 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     super.setConf(conf);
     try {
       this.zkw = zk;
-      this.replicationQueues = ReplicationFactory.getReplicationQueuesClient(zkw, conf,
-          new WarnOnlyAbortable());
+      this.replicationQueues = ReplicationFactory.getReplicationQueuesClient(
+          new ReplicationQueuesClientArguments(conf, new WarnOnlyAbortable(), zkw));
       this.replicationQueues.init();
-    } catch (ReplicationException e) {
+    } catch (Exception e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
index e472558..8d66c8f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
@@ -67,13 +68,14 @@ public class ReplicationChecker {
     try {
       this.zkw = zkw;
       this.errorReporter = errorReporter;
-      this.queuesClient = ReplicationFactory.getReplicationQueuesClient(zkw, conf, connection);
+      this.queuesClient = ReplicationFactory.getReplicationQueuesClient(
+          new ReplicationQueuesClientArguments(conf, connection, zkw));
       this.queuesClient.init();
       this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.queuesClient,
         connection);
       this.replicationPeers.init();
       this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, connection);
-    } catch (ReplicationException e) {
+    } catch (Exception e) {
       throw new IOException("failed to construct ReplicationChecker", e);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 18950a2..eecaae1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -167,7 +168,7 @@ public class TestLogsCleaner {
     ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
     cleaner.setConf(conf);
 
-    ReplicationQueuesClient rqcMock = Mockito.mock(ReplicationQueuesClient.class);
+    ReplicationQueuesClientZKImpl rqcMock = Mockito.mock(ReplicationQueuesClientZKImpl.class);
     Mockito.when(rqcMock.getQueuesZNodeCversion()).thenReturn(1, 2, 3, 4);
 
     Field rqc = ReplicationLogCleaner.class.getDeclaredField("replicationQueues");


[06/30] hbase git commit: HBASE-16016 Fix overflow deadline value in AssignmentManager#waitForAssignment (Stephen Yuan Jiang)

Posted by sy...@apache.org.
HBASE-16016 Fix overflow deadline value in AssignmentManager#waitForAssignment (Stephen Yuan Jiang)


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

Branch: refs/heads/hbase-12439
Commit: e486d274c508bea01b93f7c3175177c35482a5f1
Parents: 56c209c
Author: Stephen Yuan Jiang <sy...@gmail.com>
Authored: Mon Jun 13 18:21:41 2016 -0700
Committer: Stephen Yuan Jiang <sy...@gmail.com>
Committed: Mon Jun 13 18:21:41 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/e486d274/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 49026ac..55a15ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -1444,6 +1444,9 @@ public class AssignmentManager {
       final boolean waitTillAllAssigned, final int reassigningRegions,
       final long minEndTime) throws InterruptedException {
     long deadline = minEndTime + bulkPerRegionOpenTimeGuesstimate * (reassigningRegions + 1);
+    if (deadline < 0) { // Overflow
+      deadline = Long.MAX_VALUE; // wait forever
+    }
     return waitForAssignment(regionSet, waitTillAllAssigned, deadline);
   }
 


[14/30] hbase git commit: HBASE-16017 HBase TableOutputFormat has connection leak in getRecordWriter (Zhan Zhang)

Posted by sy...@apache.org.
HBASE-16017 HBase TableOutputFormat has connection leak in getRecordWriter (Zhan Zhang)


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

Branch: refs/heads/hbase-12439
Commit: 6c60bc9f6cf0f62c8557296dcab9335b680951b5
Parents: db234bf
Author: tedyu <yu...@gmail.com>
Authored: Wed Jun 15 06:32:23 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jun 15 06:32:23 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/mapred/TableOutputFormat.java  | 22 ++++++++++++--------
 1 file changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6c60bc9f/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
index dd72939..18b54da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
@@ -53,7 +53,7 @@ public class TableOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
    */
   protected static class TableRecordWriter implements RecordWriter<ImmutableBytesWritable, Put> {
     private BufferedMutator m_mutator;
-
+    private Connection connection;
     /**
      * Instantiate a TableRecordWriter with the HBase HClient for writing. Assumes control over the
      * lifecycle of {@code conn}.
@@ -62,8 +62,19 @@ public class TableOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
       this.m_mutator = mutator;
     }
 
+    public TableRecordWriter(JobConf job) throws IOException {
+      // expecting exactly one path
+      TableName tableName = TableName.valueOf(job.get(OUTPUT_TABLE));
+      connection = ConnectionFactory.createConnection(job);
+      m_mutator = connection.getBufferedMutator(tableName);
+    }
+
     public void close(Reporter reporter) throws IOException {
       this.m_mutator.close();
+      if (connection != null) {
+        connection.close();
+        connection = null;
+      }
     }
 
     public void write(ImmutableBytesWritable key, Put value) throws IOException {
@@ -90,14 +101,7 @@ public class TableOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
   public RecordWriter getRecordWriter(FileSystem ignored, JobConf job, String name,
       Progressable progress)
   throws IOException {
-    // expecting exactly one path
-    TableName tableName = TableName.valueOf(job.get(OUTPUT_TABLE));
-    BufferedMutator mutator =  null;
-    // Connection is not closed. Dies with JVM.  No possibility for cleanup.
-    Connection connection = ConnectionFactory.createConnection(job);
-    mutator = connection.getBufferedMutator(tableName);
-    // Clear write buffer on fail is true by default so no need to reset it.
-    return new TableRecordWriter(mutator);
+    return new TableRecordWriter(job);
   }
 
   @Override


[18/30] hbase git commit: HBASE-16034 Fix ProcedureTestingUtility#LoadCounter.setMaxProcId()

Posted by sy...@apache.org.
HBASE-16034 Fix ProcedureTestingUtility#LoadCounter.setMaxProcId()


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

Branch: refs/heads/hbase-12439
Commit: 114fe7a81e640b1068e450fa4a1cc20e22e50b3d
Parents: 2093aad
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Wed Jun 15 12:38:48 2016 -0700
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Wed Jun 15 12:38:48 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/114fe7a8/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index e3667b7..03fa516 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -296,7 +296,7 @@ public class ProcedureTestingUtility {
 
     @Override
     public void setMaxProcId(long maxProcId) {
-      maxProcId = maxProcId;
+      this.maxProcId = maxProcId;
     }
 
     @Override


[05/30] hbase git commit: HBASE-15991 CompactingMemstore#InMemoryFlushRunnable should implement Comparable/Comparator (Ram)

Posted by sy...@apache.org.
HBASE-15991 CompactingMemstore#InMemoryFlushRunnable should implement
Comparable/Comparator (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 56c209c8aaf15997888307f8a76767858cb9f470
Parents: 2d0448f
Author: Ramkrishna <ra...@intel.com>
Authored: Mon Jun 13 17:42:35 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Mon Jun 13 17:42:35 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/RegionServicesForStores.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/56c209c8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
index 72f7bf5..e481a63 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -42,7 +43,7 @@ public class RegionServicesForStores {
   private static final int POOL_SIZE = 10;
   private static final ThreadPoolExecutor INMEMORY_COMPACTION_POOL =
       new ThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 60, TimeUnit.SECONDS,
-          new StealJobQueue<Runnable>().getStealFromQueue(),
+          new LinkedBlockingQueue<Runnable>(),
           new ThreadFactory() {
             @Override
             public Thread newThread(Runnable r) {


[15/30] hbase git commit: HBASE-5291 Add Kerberos HTTP SPNEGO authentication support to HBase web consoles (Josh Elser)

Posted by sy...@apache.org.
HBASE-5291 Add Kerberos HTTP SPNEGO authentication support to HBase web consoles (Josh Elser)


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

Branch: refs/heads/hbase-12439
Commit: ae5fe1e61623dd36d33cd7abc94ca5e8229a4672
Parents: 6c60bc9
Author: tedyu <yu...@gmail.com>
Authored: Wed Jun 15 09:26:44 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jun 15 09:26:44 2016 -0700

----------------------------------------------------------------------
 hbase-server/pom.xml                            |  22 ++
 .../apache/hadoop/hbase/http/HttpServer.java    |  93 ++++++-
 .../apache/hadoop/hbase/http/InfoServer.java    |   9 +
 .../hbase/regionserver/HRegionServer.java       |   1 +
 .../hbase/http/HttpServerFunctionalTest.java    |  43 ++++
 .../hadoop/hbase/http/TestSpnegoHttpServer.java | 258 +++++++++++++++++++
 pom.xml                                         |  17 ++
 src/main/asciidoc/_chapters/security.adoc       |  53 ++++
 8 files changed, 486 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 9532d1e..8d1527f 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -572,6 +572,28 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.kerby</groupId>
+      <artifactId>kerb-client</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kerby</groupId>
+      <artifactId>kerb-simplekdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <!-- Overriden to get some SPNEGO classes only in newer version -->
+      <version>4.5.2</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <profiles>
     <!-- Needs to make the profile in apache parent pom -->

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
index 667e597..e8f875e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -27,6 +27,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
@@ -102,11 +103,30 @@ import com.sun.jersey.spi.container.servlet.ServletContainer;
 @InterfaceStability.Evolving
 public class HttpServer implements FilterContainer {
   private static final Log LOG = LogFactory.getLog(HttpServer.class);
+  private static final String EMPTY_STRING = "";
 
   static final String FILTER_INITIALIZERS_PROPERTY
       = "hbase.http.filter.initializers";
   static final String HTTP_MAX_THREADS = "hbase.http.max.threads";
 
+  public static final String HTTP_UI_AUTHENTICATION = "hbase.security.authentication.ui";
+  static final String HTTP_AUTHENTICATION_PREFIX = "hbase.security.authentication.spnego.";
+  static final String HTTP_SPNEGO_AUTHENTICATION_PREFIX = HTTP_AUTHENTICATION_PREFIX
+      + "spnego.";
+  static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX = "kerberos.principal";
+  public static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY =
+      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX;
+  static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX = "kerberos.keytab";
+  public static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY =
+      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX;
+  static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX = "kerberos.name.rules";
+  public static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY =
+      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX;
+  static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX =
+      "signature.secret.file";
+  public static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY =
+      HTTP_AUTHENTICATION_PREFIX + HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX;
+
   // The ServletContext attribute where the daemon Configuration
   // gets stored.
   public static final String CONF_CONTEXT_ATTRIBUTE = "hbase.conf";
@@ -175,6 +195,9 @@ public class HttpServer implements FilterContainer {
     // The -keypass option in keytool
     private String keyPassword;
 
+    private String kerberosNameRulesKey;
+    private String signatureSecretFileKey;
+
     @Deprecated
     private String name;
     @Deprecated
@@ -302,6 +325,16 @@ public class HttpServer implements FilterContainer {
       return this;
     }
 
+    public Builder setKerberosNameRulesKey(String kerberosNameRulesKey) {
+      this.kerberosNameRulesKey = kerberosNameRulesKey;
+      return this;
+    }
+
+    public Builder setSignatureSecretFileKey(String signatureSecretFileKey) {
+      this.signatureSecretFileKey = signatureSecretFileKey;
+      return this;
+    }
+
     public Builder setAppDir(String appDir) {
         this.appDir = appDir;
         return this;
@@ -344,7 +377,8 @@ public class HttpServer implements FilterContainer {
       HttpServer server = new HttpServer(this);
 
       if (this.securityEnabled) {
-        server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey);
+        server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey, kerberosNameRulesKey,
+            signatureSecretFileKey);
       }
 
       if (connector != null) {
@@ -927,21 +961,60 @@ public class HttpServer implements FilterContainer {
   }
 
   private void initSpnego(Configuration conf, String hostName,
-      String usernameConfKey, String keytabConfKey) throws IOException {
+      String usernameConfKey, String keytabConfKey, String kerberosNameRuleKey,
+      String signatureSecretKeyFileKey) throws IOException {
     Map<String, String> params = new HashMap<String, String>();
-    String principalInConf = conf.get(usernameConfKey);
-    if (principalInConf != null && !principalInConf.isEmpty()) {
-      params.put("kerberos.principal", SecurityUtil.getServerPrincipal(
+    String principalInConf = getOrEmptyString(conf, usernameConfKey);
+    if (!principalInConf.isEmpty()) {
+      params.put(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX, SecurityUtil.getServerPrincipal(
           principalInConf, hostName));
     }
-    String httpKeytab = conf.get(keytabConfKey);
-    if (httpKeytab != null && !httpKeytab.isEmpty()) {
-      params.put("kerberos.keytab", httpKeytab);
+    String httpKeytab = getOrEmptyString(conf, keytabConfKey);
+    if (!httpKeytab.isEmpty()) {
+      params.put(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX, httpKeytab);
+    }
+    String kerberosNameRule = getOrEmptyString(conf, kerberosNameRuleKey);
+    if (!kerberosNameRule.isEmpty()) {
+      params.put(HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX, kerberosNameRule);
+    }
+    String signatureSecretKeyFile = getOrEmptyString(conf, signatureSecretKeyFileKey);
+    if (!signatureSecretKeyFile.isEmpty()) {
+      params.put(HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX,
+          signatureSecretKeyFile);
     }
     params.put(AuthenticationFilter.AUTH_TYPE, "kerberos");
 
-    defineFilter(webAppContext, SPNEGO_FILTER,
-                 AuthenticationFilter.class.getName(), params, null);
+    // Verify that the required options were provided
+    if (isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX)) ||
+            isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX))) {
+      throw new IllegalArgumentException(usernameConfKey + " and "
+          + keytabConfKey + " are both required in the configuration "
+          + "to enable SPNEGO/Kerberos authentication for the Web UI");
+    }
+
+    addGlobalFilter(SPNEGO_FILTER, AuthenticationFilter.class.getName(), params);
+  }
+
+  /**
+   * Returns true if the argument is non-null and not whitespace
+   */
+  private boolean isMissing(String value) {
+    if (null == value) {
+      return true;
+    }
+    return value.trim().isEmpty();
+  }
+
+  /**
+   * Extracts the value for the given key from the configuration of returns a string of
+   * zero length.
+   */
+  private String getOrEmptyString(Configuration conf, String key) {
+    if (null == key) {
+      return EMPTY_STRING;
+    }
+    final String value = conf.get(key.trim());
+    return null == value ? EMPTY_STRING : value;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
index 5ff6370..0f6c3dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
@@ -75,6 +75,15 @@ public class InfoServer {
         HBaseConfiguration.getPassword(c, "ssl.server.truststore.password", null),
         c.get("ssl.server.truststore.type", "jks"));
     }
+    // Enable SPNEGO authentication
+    if ("kerberos".equalsIgnoreCase(c.get(HttpServer.HTTP_UI_AUTHENTICATION, null))) {
+      builder.setUsernameConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY)
+        .setKeytabConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY)
+        .setKerberosNameRulesKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY)
+        .setSignatureSecretFileKey(
+            HttpServer.HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY)
+        .setSecurityEnabled(true);
+    }
     this.httpServer = builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/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 b97c23f..45a1095 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
@@ -104,6 +104,7 @@ import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.http.HttpServer;
 import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
index 5c832be..7e773b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.http.HttpServer.Builder;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.ServerSocket;
 import java.net.URI;
 import java.net.URL;
 import java.net.MalformedURLException;
@@ -94,6 +95,15 @@ public class HttpServerFunctionalTest extends Assert {
     return createServer(TEST, conf, pathSpecs);
   }
 
+  public static HttpServer createTestServerWithSecurity(Configuration conf) throws IOException {
+    prepareTestWebapp();
+    return localServerBuilder(TEST).setFindPort(true).setConf(conf).setSecurityEnabled(true)
+        // InfoServer normally sets these for us
+        .setUsernameConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY)
+        .setKeytabConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY)
+        .build();
+  }
+
   /**
    * Prepare the test webapp by creating the directory from the test properties
    * fail if the directory cannot be created.
@@ -226,4 +236,37 @@ public class HttpServerFunctionalTest extends Assert {
     }
     return out.toString();
   }
+
+  /**
+   * Recursively deletes a {@link File}.
+   */
+  protected static void deleteRecursively(File d) {
+    if (d.isDirectory()) {
+      for (String name : d.list()) {
+        File child = new File(d, name);
+        if (child.isFile()) {
+          child.delete();
+        } else {
+          deleteRecursively(d);
+        }
+      }
+    }
+    d.delete();
+  }
+
+  /**
+   * Picks a free port on the host by binding a Socket to '0'.
+   */
+  protected static int getFreePort() throws IOException {
+    ServerSocket s = new ServerSocket(0);
+    try {
+      s.setReuseAddress(true);
+      int port = s.getLocalPort();
+      return port;
+    } finally {
+      if (null != s) {
+        s.close();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
new file mode 100644
index 0000000..a84895f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
@@ -0,0 +1,258 @@
+/*
+ * 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.http;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+import java.util.Set;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosTicket;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.http.TestHttpServer.EchoServlet;
+import org.apache.hadoop.hbase.http.resource.JerseyResource;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.http.HttpHost;
+import org.apache.http.HttpResponse;
+import org.apache.http.auth.AuthSchemeProvider;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.KerberosCredentials;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.config.AuthSchemes;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.config.Lookup;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.entity.ContentType;
+import org.apache.http.impl.auth.SPNegoSchemeFactory;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.kerby.kerberos.kerb.KrbException;
+import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
+import org.ietf.jgss.GSSCredential;
+import org.ietf.jgss.GSSManager;
+import org.ietf.jgss.GSSName;
+import org.ietf.jgss.Oid;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for SPNEGO authentication on the HttpServer. Uses Kerby's MiniKDC and Apache
+ * HttpComponents to verify that a simple Servlet is reachable via SPNEGO and unreachable w/o.
+ */
+@Category({MiscTests.class, SmallTests.class})
+public class TestSpnegoHttpServer extends HttpServerFunctionalTest {
+  private static final Log LOG = LogFactory.getLog(TestSpnegoHttpServer.class);
+  private static final String KDC_SERVER_HOST = "localhost";
+  private static final String CLIENT_PRINCIPAL = "client";
+
+  private static HttpServer server;
+  private static URL baseUrl;
+  private static SimpleKdcServer kdc;
+  private static File infoServerKeytab;
+  private static File clientKeytab;
+
+  @BeforeClass
+  public static void setupServer() throws Exception {
+    final String serverPrincipal = "HTTP/" + KDC_SERVER_HOST;
+    final File target = new File(System.getProperty("user.dir"), "target");
+    assertTrue(target.exists());
+
+    kdc = buildMiniKdc();
+    kdc.start();
+
+    File keytabDir = new File(target, TestSpnegoHttpServer.class.getSimpleName()
+        + "_keytabs");
+    if (keytabDir.exists()) {
+      deleteRecursively(keytabDir);
+    }
+    keytabDir.mkdirs();
+
+    infoServerKeytab = new File(keytabDir, serverPrincipal.replace('/', '_') + ".keytab");
+    clientKeytab = new File(keytabDir, CLIENT_PRINCIPAL + ".keytab");
+
+    setupUser(kdc, clientKeytab, CLIENT_PRINCIPAL);
+    setupUser(kdc, infoServerKeytab, serverPrincipal);
+
+    Configuration conf = buildSpnegoConfiguration(serverPrincipal, infoServerKeytab);
+
+    server = createTestServerWithSecurity(conf);
+    server.addServlet("echo", "/echo", EchoServlet.class);
+    server.addJerseyResourcePackage(JerseyResource.class.getPackage().getName(), "/jersey/*");
+    server.start();
+    baseUrl = getServerURL(server);
+
+    LOG.info("HTTP server started: "+ baseUrl);
+  }
+
+  @AfterClass
+  public static void stopServer() throws Exception {
+    try {
+      if (null != server) {
+        server.stop();
+      }
+    } catch (Exception e) {
+      LOG.info("Failed to stop info server", e);
+    }
+    try {
+      if (null != kdc) {
+        kdc.stop();
+      }
+    } catch (Exception e) {
+      LOG.info("Failed to stop mini KDC", e);
+    }
+  }
+
+  private static void setupUser(SimpleKdcServer kdc, File keytab, String principal)
+      throws KrbException {
+    kdc.createPrincipal(principal);
+    kdc.exportPrincipal(principal, keytab);
+  }
+
+  private static SimpleKdcServer buildMiniKdc() throws Exception {
+    SimpleKdcServer kdc = new SimpleKdcServer();
+
+    final File target = new File(System.getProperty("user.dir"), "target");
+    File kdcDir = new File(target, TestSpnegoHttpServer.class.getSimpleName());
+    if (kdcDir.exists()) {
+      deleteRecursively(kdcDir);
+    }
+    kdcDir.mkdirs();
+    kdc.setWorkDir(kdcDir);
+
+    kdc.setKdcHost(KDC_SERVER_HOST);
+    int kdcPort = getFreePort();
+    kdc.setAllowTcp(true);
+    kdc.setAllowUdp(false);
+    kdc.setKdcTcpPort(kdcPort);
+
+    LOG.info("Starting KDC server at " + KDC_SERVER_HOST + ":" + kdcPort);
+
+    kdc.init();
+
+    return kdc;
+  }
+
+  private static Configuration buildSpnegoConfiguration(String serverPrincipal, File
+      serverKeytab) {
+    Configuration conf = new Configuration();
+    KerberosName.setRules("DEFAULT");
+
+    conf.setInt(HttpServer.HTTP_MAX_THREADS, 10);
+
+    // Enable Kerberos (pre-req)
+    conf.set("hbase.security.authentication", "kerberos");
+    conf.set(HttpServer.HTTP_UI_AUTHENTICATION, "kerberos");
+    conf.set(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY, serverPrincipal);
+    conf.set(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY, serverKeytab.getAbsolutePath());
+
+    return conf;
+  }
+
+  @Test
+  public void testUnauthorizedClientsDisallowed() throws IOException {
+    URL url = new URL(getServerURL(server), "/echo?a=b");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
+  }
+
+  @Test
+  public void testAllowedClient() throws Exception {
+    // Create the subject for the client
+    final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(CLIENT_PRINCIPAL, clientKeytab);
+    final Set<Principal> clientPrincipals = clientSubject.getPrincipals();
+    // Make sure the subject has a principal
+    assertFalse(clientPrincipals.isEmpty());
+
+    // Get a TGT for the subject (might have many, different encryption types). The first should
+    // be the default encryption type.
+    Set<KerberosTicket> privateCredentials =
+            clientSubject.getPrivateCredentials(KerberosTicket.class);
+    assertFalse(privateCredentials.isEmpty());
+    KerberosTicket tgt = privateCredentials.iterator().next();
+    assertNotNull(tgt);
+
+    // The name of the principal
+    final String principalName = clientPrincipals.iterator().next().getName();
+
+    // Run this code, logged in as the subject (the client)
+    HttpResponse resp = Subject.doAs(clientSubject,
+        new PrivilegedExceptionAction<HttpResponse>() {
+      @Override
+      public HttpResponse run() throws Exception {
+        // Logs in with Kerberos via GSS
+        GSSManager gssManager = GSSManager.getInstance();
+        // jGSS Kerberos login constant
+        Oid oid = new Oid("1.2.840.113554.1.2.2");
+        GSSName gssClient = gssManager.createName(principalName, GSSName.NT_USER_NAME);
+        GSSCredential credential = gssManager.createCredential(gssClient,
+            GSSCredential.DEFAULT_LIFETIME, oid, GSSCredential.INITIATE_ONLY);
+
+        HttpClientContext context = HttpClientContext.create();
+        Lookup<AuthSchemeProvider> authRegistry = RegistryBuilder.<AuthSchemeProvider>create()
+            .register(AuthSchemes.SPNEGO, new SPNegoSchemeFactory(true, true))
+            .build();
+
+        HttpClient client = HttpClients.custom().setDefaultAuthSchemeRegistry(authRegistry).build();
+        BasicCredentialsProvider credentialsProvider = new BasicCredentialsProvider();
+        credentialsProvider.setCredentials(AuthScope.ANY, new KerberosCredentials(credential));
+
+        URL url = new URL(getServerURL(server), "/echo?a=b");
+        context.setTargetHost(new HttpHost(url.getHost(), url.getPort()));
+        context.setCredentialsProvider(credentialsProvider);
+        context.setAuthSchemeRegistry(authRegistry);
+
+        HttpGet get = new HttpGet(url.toURI());
+        return client.execute(get, context);
+      }
+    });
+
+    assertNotNull(resp);
+    assertEquals(HttpURLConnection.HTTP_OK, resp.getStatusLine().getStatusCode());
+    assertEquals("a:b", EntityUtils.toString(resp.getEntity()).trim());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMissingConfigurationThrowsException() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(HttpServer.HTTP_MAX_THREADS, 10);
+    // Enable Kerberos (pre-req)
+    conf.set("hbase.security.authentication", "kerberos");
+    // Intentionally skip keytab and principal
+
+    HttpServer customServer = createTestServerWithSecurity(conf);
+    customServer.addServlet("echo", "/echo", EchoServlet.class);
+    customServer.addJerseyResourcePackage(JerseyResource.class.getPackage().getName(), "/jersey/*");
+    customServer.start();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 93b7a0e..b34401d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1231,6 +1231,7 @@
     <!-- Do not use versions earlier than 3.2.2 due to a security vulnerability -->
     <collections.version>3.2.2</collections.version>
     <httpclient.version>4.3.6</httpclient.version>
+    <httpcore.version>4.4.4</httpcore.version>
     <metrics-core.version>3.1.2</metrics-core.version>
     <guava.version>12.0.1</guava.version>
     <jsr305.version>1.3.9</jsr305.version>
@@ -1260,6 +1261,7 @@
     <jcodings.version>1.0.8</jcodings.version>
     <spy.version>2.11.6</spy.version>
     <bouncycastle.version>1.46</bouncycastle.version>
+    <kerby.version>1.0.0-RC2</kerby.version>
     <!-- Plugin Dependencies -->
     <maven.assembly.version>2.4</maven.assembly.version>
     <maven.antrun.version>1.8</maven.antrun.version>
@@ -1525,6 +1527,11 @@
         <version>${httpclient.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.httpcomponents</groupId>
+        <artifactId>httpcore</artifactId>
+        <version>${httpcore.version}</version>
+      </dependency>
+      <dependency>
         <groupId>commons-cli</groupId>
         <artifactId>commons-cli</artifactId>
         <version>${commons-cli.version}</version>
@@ -1806,6 +1813,16 @@
        <version>${bouncycastle.version}</version>
        <scope>test</scope>
      </dependency>
+     <dependency>
+       <groupId>org.apache.kerby</groupId>
+       <artifactId>kerb-client</artifactId>
+       <version>${kerby.version}</version>
+     </dependency>
+     <dependency>
+       <groupId>org.apache.kerby</groupId>
+       <artifactId>kerb-simplekdc</artifactId>
+       <version>${kerby.version}</version>
+     </dependency>
     </dependencies>
   </dependencyManagement>
   <!-- Dependencies needed by subprojects -->

http://git-wip-us.apache.org/repos/asf/hbase/blob/ae5fe1e6/src/main/asciidoc/_chapters/security.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/security.adoc b/src/main/asciidoc/_chapters/security.adoc
index 0d1407a..85e503c 100644
--- a/src/main/asciidoc/_chapters/security.adoc
+++ b/src/main/asciidoc/_chapters/security.adoc
@@ -69,6 +69,59 @@ See Nick Dimiduk's contribution on this link:http://stackoverflow.com/questions/
 If you know how to fix this without opening a second port for HTTPS, patches are appreciated.
 ====
 
+[[hbase.secure.spnego.ui]]
+== Using SPNEGO for Kerberos authentication with Web UIs
+
+Kerberos-authentication to HBase Web UIs can be enabled via configuring SPNEGO with the `hbase.security.authentication.ui`
+property in _hbase-site.xml_. Enabling this authentication requires that HBase is also configured to use Kerberos authentication
+for RPCs (e.g `hbase.security.authentication` = `kerberos`).
+
+[source,xml]
+----
+<property>
+  <name>hbase.security.authentication.ui</name>
+  <value>kerberos</value>
+  <description>Controls what kind of authentication should be used for the HBase web UIs.</description>
+</property>
+<property>
+  <name>hbase.security.authentication</name>
+  <value>kerberos</value>
+  <description>The Kerberos keytab file to use for SPNEGO authentication by the web server.</description>
+</property>
+----
+
+A number of properties exist to configure SPNEGO authentication for the web server:
+
+[source,xml]
+----
+<property>
+  <name>hbase.security.authentication.spnego.kerberos.principal</name>
+  <value>HTTP/_HOST@EXAMPLE.COM</value>
+  <description>Required for SPNEGO, the Kerberos principal to use for SPNEGO authentication by the
+  web server. The _HOST keyword will be automatically substituted with the node's
+  hostname.</description>
+</property>
+<property>
+  <name>hbase.security.authentication.spnego.kerberos.keytab</name>
+  <value>/etc/security/keytabs/spnego.service.keytab</value>
+  <description>Required for SPNEGO, the Kerberos keytab file to use for SPNEGO authentication by the
+  web server.</description>
+</property>
+<property>
+  <name>hbase.security.authentication.spnego.kerberos.name.rules</name>
+  <value></value>
+  <description>Optional, Hadoop-style `auth_to_local` rules which will be parsed and used in the
+  handling of Kerberos principals</description>
+</property>
+<property>
+  <name>hbase.security.authentication.signature.secret.file</name>
+  <value></value>
+  <description>Optional, a file whose contents will be used as a secret to sign the HTTP cookies
+  as a part of the SPNEGO authentication handshake. If this is not provided, Java's `Random` library
+  will be used for the secret.</description>
+</property>
+----
+
 [[hbase.secure.configuration]]
 == Secure Client Access to Apache HBase
 


[19/30] hbase git commit: HBASE-16026 Master UI should display status of additional ZK switches

Posted by sy...@apache.org.
HBASE-16026 Master UI should display status of additional ZK switches


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

Branch: refs/heads/hbase-12439
Commit: 5f9d1a71c8b920631062b7a30c1e73a8da5e6988
Parents: 114fe7a
Author: Mikhail Antonov <an...@apache.org>
Authored: Wed Jun 15 12:58:41 2016 -0700
Committer: Mikhail Antonov <an...@apache.org>
Committed: Wed Jun 15 12:59:24 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon  | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5f9d1a71/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index e39ad8a..056c2d7 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -39,6 +39,7 @@ org.apache.hadoop.hbase.ServerLoad;
 org.apache.hadoop.hbase.ServerName;
 org.apache.hadoop.hbase.TableName;
 org.apache.hadoop.hbase.client.Admin;
+org.apache.hadoop.hbase.client.MasterSwitchType;
 org.apache.hadoop.hbase.client.SnapshotDescription;
 org.apache.hadoop.hbase.master.AssignmentManager;
 org.apache.hadoop.hbase.master.DeadServer;
@@ -168,6 +169,20 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
             is only expected to be disabled during rolling upgrade scenarios.
           </div>
         </%if>
+        <%if !master.isSplitOrMergeEnabled(MasterSwitchType.SPLIT) %>
+          <div class="alert alert-warning">
+            Region splits are disabled. This may be the result of HBCK aborting while
+            running in repair mode. Manually enable splits from the HBase shell,
+            or re-run HBCK in repair mode.
+          </div>
+        </%if>
+        <%if !master.isSplitOrMergeEnabled(MasterSwitchType.MERGE) %>
+          <div class="alert alert-warning">
+            Region merges are disabled. This may be the result of HBCK aborting while
+            running in repair mode. Manually enable merges from the HBase shell,
+            or re-run HBCK in repair mode.
+          </div>
+        </%if>
 
         <section>
             <h2>Region Servers</h2>


[26/30] hbase git commit: HBASE-16033 Add more details in logging of responseTooSlow/TooLarge

Posted by sy...@apache.org.
HBASE-16033 Add more details in logging of responseTooSlow/TooLarge


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

Branch: refs/heads/hbase-12439
Commit: 6d02f36ac78426f178b977566c170fb2e4b3503a
Parents: d1de933
Author: Yu Li <li...@apache.org>
Authored: Thu Jun 16 16:40:38 2016 +0800
Committer: Yu Li <li...@apache.org>
Committed: Thu Jun 16 16:42:10 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 35 ++++----------------
 1 file changed, 6 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6d02f36a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index c7f5a10..ad88e8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -2235,7 +2235,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       if (tooSlow || tooLarge) {
         // when tagging, we let TooLarge trump TooSmall to keep output simple
         // note that large responses will often also be slow.
-        logResponse(new Object[]{param},
+        logResponse(param,
             md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
             (tooLarge ? "TooLarge" : "TooSlow"),
             status.getClient(), startTime, processingTime, qTime,
@@ -2261,7 +2261,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   /**
    * Logs an RPC response to the LOG file, producing valid JSON objects for
    * client Operations.
-   * @param params The parameters received in the call.
+   * @param param The parameters received in the call.
    * @param methodName The name of the method invoked
    * @param call The string representation of the call
    * @param tag  The tag that will be used to indicate this event in the log.
@@ -2272,7 +2272,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
    *                        prior to being initiated, in ms.
    * @param responseSize    The size in bytes of the response buffer.
    */
-  void logResponse(Object[] params, String methodName, String call, String tag,
+  void logResponse(Message param, String methodName, String call, String tag,
       String clientAddress, long startTime, int processingTime, int qTime,
       long responseSize)
           throws IOException {
@@ -2285,32 +2285,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     responseInfo.put("client", clientAddress);
     responseInfo.put("class", server == null? "": server.getClass().getSimpleName());
     responseInfo.put("method", methodName);
-    if (params.length == 2 && server instanceof HRegionServer &&
-        params[0] instanceof byte[] &&
-        params[1] instanceof Operation) {
-      // if the slow process is a query, we want to log its table as well
-      // as its own fingerprint
-      TableName tableName = TableName.valueOf(
-          HRegionInfo.parseRegionName((byte[]) params[0])[0]);
-      responseInfo.put("table", tableName.getNameAsString());
-      // annotate the response map with operation details
-      responseInfo.putAll(((Operation) params[1]).toMap());
-      // report to the log file
-      LOG.warn("(operation" + tag + "): " +
-               MAPPER.writeValueAsString(responseInfo));
-    } else if (params.length == 1 && server instanceof HRegionServer &&
-        params[0] instanceof Operation) {
-      // annotate the response map with operation details
-      responseInfo.putAll(((Operation) params[0]).toMap());
-      // report to the log file
-      LOG.warn("(operation" + tag + "): " +
-               MAPPER.writeValueAsString(responseInfo));
-    } else {
-      // can't get JSON details, so just report call.toString() along with
-      // a more generic tag.
-      responseInfo.put("call", call);
-      LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
-    }
+    responseInfo.put("call", call);
+    responseInfo.put("param", ProtobufUtil.getShortTextFormat(param));
+    LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
   }
 
   /** Stops the service.  No new calls will be handled after this is called. */


[11/30] hbase git commit: HBASE-15971 Regression: Random Read/WorkloadC slower in 1.x than 0.98 Restore FIFO as the default in place of 'deadline' that sorted on request priority. The sort costs us throughput. Enable it if you want it rather than have it

Posted by sy...@apache.org.
HBASE-15971 Regression: Random Read/WorkloadC slower in 1.x than 0.98 Restore FIFO as the default in place of 'deadline' that sorted on request priority. The sort costs us throughput. Enable it if you want it rather than have it as default.

Name threads to include the scheduler type.


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

Branch: refs/heads/hbase-12439
Commit: fa50d456a8a05517004fec853258e4a01ca35a23
Parents: b69c77a
Author: stack <st...@apache.org>
Authored: Mon Jun 13 15:26:13 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Jun 14 16:51:24 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    | 26 +++++++++++---------
 1 file changed, 15 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fa50d456/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index d9d61c1..85bf78d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -52,12 +52,16 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
   public static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY =
       "hbase.ipc.server.callqueue.handler.factor";
 
-  /** If set to 'deadline', the default, uses a priority queue and deprioritizes long-running scans
+  /**
+   * The default, 'fifo', has the least friction but is dumb.
+   * If set to 'deadline', uses a priority queue and deprioritizes long-running scans. Sorting by
+   * priority comes at a cost, reduced throughput.
    */
-  public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
   public static final String CALL_QUEUE_TYPE_CODEL_CONF_VALUE = "codel";
   public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
   public static final String CALL_QUEUE_TYPE_FIFO_CONF_VALUE = "fifo";
+  public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
+  public static final String CALL_QUEUE_TYPE_CONF_DEFAULT = CALL_QUEUE_TYPE_FIFO_CONF_VALUE;
 
   /** max delay in msec used to bound the deprioritized requests */
   public static final String QUEUE_MAX_CALL_DELAY_CONF_KEY
@@ -181,7 +185,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     this.abortable = server;
 
     String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY,
-      CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
+        CALL_QUEUE_TYPE_FIFO_CONF_VALUE);
     float callqReadShare = conf.getFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0);
     float callqScanShare = conf.getFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0);
 
@@ -200,18 +204,18 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
       // multiple read/write queues
       if (isDeadlineQueueType(callQueueType)) {
         CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
-        callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
+        callExecutor = new RWQueueRpcExecutor("RW.deadline.Q", handlerCount, numCallQueues,
             callqReadShare, callqScanShare, maxQueueLength, conf, abortable,
             BoundedPriorityBlockingQueue.class, callPriority);
       } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
         Object[] callQueueInitArgs = {maxQueueLength, codelTargetDelay, codelInterval,
           codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches};
-        callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount,
+        callExecutor = new RWQueueRpcExecutor("RW.codel.Q", handlerCount,
           numCallQueues, callqReadShare, callqScanShare,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs);
       } else {
-        callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
+        callExecutor = new RWQueueRpcExecutor("RW.fifo.Q", handlerCount, numCallQueues,
           callqReadShare, callqScanShare, maxQueueLength, conf, abortable);
       }
     } else {
@@ -219,26 +223,26 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
       if (isDeadlineQueueType(callQueueType)) {
         CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
         callExecutor =
-          new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
+          new BalancedQueueRpcExecutor("B.deadline.Q", handlerCount, numCallQueues,
             conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
       } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
         callExecutor =
-          new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
+          new BalancedQueueRpcExecutor("B.codel.Q", handlerCount, numCallQueues,
             conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
             codelTargetDelay, codelInterval, codelLifoThreshold,
             numGeneralCallsDropped, numLifoModeSwitches);
       } else {
-        callExecutor = new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount,
+        callExecutor = new BalancedQueueRpcExecutor("B.fifo.Q", handlerCount,
             numCallQueues, maxQueueLength, conf, abortable);
       }
     }
     // Create 2 queues to help priorityExecutor be more scalable.
     this.priorityExecutor = priorityHandlerCount > 0 ?
-      new BalancedQueueRpcExecutor("BalancedQ.priority", priorityHandlerCount, 2,
+      new BalancedQueueRpcExecutor("B.priority.fifo.Q", priorityHandlerCount, 2,
           maxPriorityQueueLength):
       null;
    this.replicationExecutor =
-     replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("BalancedQ.replication",
+     replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("B.replication.fifo.Q",
        replicationHandlerCount, 1, maxQueueLength, conf, abortable) : null;
   }
 


[08/30] hbase git commit: HBASE-15746 Remove extra RegionCoprocessor preClose() in RSRpcServices#closeRegion (Stephen Yuan Jiang)

Posted by sy...@apache.org.
HBASE-15746 Remove extra RegionCoprocessor preClose() in RSRpcServices#closeRegion (Stephen Yuan Jiang)


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

Branch: refs/heads/hbase-12439
Commit: df2400435340fe4d81cc199a4fd7827fc7467fe2
Parents: 17bcf14
Author: Stephen Yuan Jiang <sy...@gmail.com>
Authored: Tue Jun 14 13:30:41 2016 -0700
Committer: Stephen Yuan Jiang <sy...@gmail.com>
Committed: Tue Jun 14 13:30:41 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/HRegionServer.java    | 1 +
 .../org/apache/hadoop/hbase/regionserver/RSRpcServices.java    | 6 ------
 2 files changed, 1 insertion(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/df240043/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 396e182..b97c23f 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
@@ -2845,6 +2845,7 @@ public class HRegionServer extends HasThread implements
       throws NotServingRegionException {
     //Check for permissions to close.
     Region actualRegion = this.getFromOnlineRegions(encodedName);
+    // Can be null if we're calling close on a region that's not online
     if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
       try {
         actualRegion.getCoprocessorHost().preClose(false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/df240043/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 e09e0e3..f45fc69 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
@@ -1324,12 +1324,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       }
       final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion());
 
-      // Can be null if we're calling close on a region that's not online
-      final Region region = regionServer.getFromOnlineRegions(encodedRegionName);
-      if ((region  != null) && (region .getCoprocessorHost() != null)) {
-        region.getCoprocessorHost().preClose(false);
-      }
-
       requestCount.increment();
       LOG.info("Close " + encodedRegionName + ", moving to " + sn);
       boolean closed = regionServer.closeRegion(encodedRegionName, false, sn);


[03/30] hbase git commit: HBASE-15991 CompactingMemstore#InMemoryFlushRunnable should implement Comparable/Comparator (Ram)

Posted by sy...@apache.org.
HBASE-15991 CompactingMemstore#InMemoryFlushRunnable should implement
Comparable/Comparator (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 2738518d439e881ff57622b10ba835dbaab25c77
Parents: 6360cc0
Author: Ramkrishna <ra...@intel.com>
Authored: Mon Jun 13 12:43:52 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Mon Jun 13 12:43:52 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/RegionServicesForStores.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2738518d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
index 72f7bf5..e481a63 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -42,7 +43,7 @@ public class RegionServicesForStores {
   private static final int POOL_SIZE = 10;
   private static final ThreadPoolExecutor INMEMORY_COMPACTION_POOL =
       new ThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 60, TimeUnit.SECONDS,
-          new StealJobQueue<Runnable>().getStealFromQueue(),
+          new LinkedBlockingQueue<Runnable>(),
           new ThreadFactory() {
             @Override
             public Thread newThread(Runnable r) {


[25/30] hbase git commit: HBASE-16031 Documents about "hbase.replication" default value seems wrong

Posted by sy...@apache.org.
HBASE-16031 Documents about "hbase.replication" default value seems wrong


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

Branch: refs/heads/hbase-12439
Commit: d1de9337ef7084c6d23d0bc3116d29049e8af255
Parents: f19f1d9
Author: chenheng <ch...@apache.org>
Authored: Thu Jun 16 13:52:16 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Thu Jun 16 14:12:37 2016 +0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/ops_mgt.adoc | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d1de9337/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index bc75951..e7ca940 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -1651,9 +1651,9 @@ The following metrics are exposed at the global region server level and (since H
 | hbase.replication
 | Whether replication is enabled or disabled on a given
                 cluster
-| false
+| true
 
-| eplication.sleep.before.failover
+| replication.sleep.before.failover
 | How many milliseconds a worker should sleep before attempting to replicate
                 a dead region server's WAL queues.
 |


[30/30] hbase git commit: HBASE-15429 Add split policy for busy regions

Posted by sy...@apache.org.
HBASE-15429 Add split policy for busy regions

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


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

Branch: refs/heads/hbase-12439
Commit: 3abd52bdc6db926d930fd94cfce5bd7ba6fd005f
Parents: 6d0e0e3
Author: Ashu Pachauri <as...@gmail.com>
Authored: Mon Apr 11 07:43:06 2016 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Thu Jun 16 17:36:54 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        |   2 +-
 .../regionserver/BusyRegionSplitPolicy.java     | 157 +++++++++++++++++++
 .../regionserver/TestRegionSplitPolicy.java     |  51 ++++++
 3 files changed, 209 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3abd52bd/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 55ac497..6010b7a 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -322,7 +322,7 @@ possible configurations would overwhelm and obscure the important.
     <value>org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy</value>
     <description>
       A split policy determines when a region should be split. The various other split policies that
-      are available currently are ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy,
+      are available currently are BusyRegionSplitPolicy, ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy,
       DelimitedKeyPrefixRegionSplitPolicy, and KeyPrefixRegionSplitPolicy.
       DisabledRegionSplitPolicy blocks manual region splitting.
     </description>

http://git-wip-us.apache.org/repos/asf/hbase/blob/3abd52bd/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BusyRegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BusyRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BusyRegionSplitPolicy.java
new file mode 100644
index 0000000..81c468f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BusyRegionSplitPolicy.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * This class represents a split policy which makes the split decision based
+ * on how busy a region is. The metric that is used here is the fraction of
+ * total write requests that are blocked due to high memstore utilization.
+ * This fractional rate is calculated over a running window of
+ * "hbase.busy.policy.aggWindow" milliseconds. The rate is a time-weighted
+ * aggregated average of the rate in the current window and the
+ * true average rate in the previous window.
+ *
+ */
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BusyRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPolicy {
+
+  private static final Log LOG = LogFactory.getLog(BusyRegionSplitPolicy.class);
+
+  // Maximum fraction blocked write requests before region is considered for split
+  private float maxBlockedRequests;
+  public static final float DEFAULT_MAX_BLOCKED_REQUESTS = 0.2f;
+
+  // Minimum age of the region in milliseconds before it is considered for split
+  private long minAge = -1;
+  public static final long DEFAULT_MIN_AGE_MS = 600000;  // 10 minutes
+
+  // The window time in milliseconds over which the blocked requests rate is calculated
+  private long aggregationWindow;
+  public static final long DEFAULT_AGGREGATION_WINDOW = 300000;  // 5 minutes
+
+  private HRegion region;
+  private long prevTime;
+  private long startTime;
+  private long writeRequestCount;
+  private long blockedRequestCount;
+  private float blockedRate;
+
+  @Override
+  protected void configureForRegion(final HRegion region) {
+    super.configureForRegion(region);
+    this.region = region;
+    Configuration conf = getConf();
+
+    maxBlockedRequests = conf.getFloat("hbase.busy.policy.blockedRequests",
+        DEFAULT_MAX_BLOCKED_REQUESTS);
+    minAge = conf.getLong("hbase.busy.policy.minAge", DEFAULT_MIN_AGE_MS);
+    aggregationWindow = conf.getLong("hbase.busy.policy.aggWindow",
+        DEFAULT_AGGREGATION_WINDOW);
+
+    if (maxBlockedRequests < 0.00001f || maxBlockedRequests > 0.99999f) {
+      LOG.warn("Threshold for maximum blocked requests is set too low or too high, "
+          + " resetting to default of " + DEFAULT_MAX_BLOCKED_REQUESTS);
+      maxBlockedRequests = DEFAULT_MAX_BLOCKED_REQUESTS;
+    }
+
+    if (aggregationWindow <= 0) {
+      LOG.warn("Aggregation window size is too low: " + aggregationWindow
+          + ". Resetting it to default of " + DEFAULT_AGGREGATION_WINDOW);
+      aggregationWindow = DEFAULT_AGGREGATION_WINDOW;
+    }
+
+    init();
+  }
+
+  private synchronized void init() {
+    startTime = EnvironmentEdgeManager.currentTime();
+    prevTime = startTime;
+    blockedRequestCount = region.getBlockedRequestsCount();
+    writeRequestCount = region.getWriteRequestsCount();
+  }
+
+  @Override
+  protected boolean shouldSplit() {
+    float blockedReqRate = updateRate();
+    if (super.shouldSplit()) {
+      return true;
+    }
+
+    if (EnvironmentEdgeManager.currentTime() <  startTime + minAge) {
+      return false;
+    }
+
+    for (Store store: region.getStores()) {
+      if (!store.canSplit()) {
+        return false;
+      }
+    }
+
+    if (blockedReqRate >= maxBlockedRequests) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Going to split region " + region.getRegionInfo().getRegionNameAsString()
+            + " because it's too busy. Blocked Request rate: " + blockedReqRate);
+      }
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Update the blocked request rate based on number of blocked and total write requests in the
+   * last aggregation window, or since last call to this method, whichever is farthest in time.
+   * Uses weighted rate calculation based on the previous rate and new data.
+   *
+   * @return Updated blocked request rate.
+   */
+  private synchronized float updateRate() {
+    float aggBlockedRate;
+    long curTime = EnvironmentEdgeManager.currentTime();
+
+    long newBlockedReqs = region.getBlockedRequestsCount();
+    long newWriteReqs = region.getWriteRequestsCount();
+
+    aggBlockedRate =
+        (newBlockedReqs - blockedRequestCount) / (newWriteReqs - writeRequestCount + 0.00001f);
+
+    if (curTime - prevTime >= aggregationWindow) {
+      blockedRate = aggBlockedRate;
+      prevTime = curTime;
+      blockedRequestCount = newBlockedReqs;
+      writeRequestCount = newWriteReqs;
+    } else if (curTime - startTime >= aggregationWindow) {
+      // Calculate the aggregate blocked rate as the weighted sum of
+      // previous window's average blocked rate and blocked rate in this window so far.
+      float timeSlice = (curTime - prevTime) / (aggregationWindow + 0.0f);
+      aggBlockedRate = (1 - timeSlice) * blockedRate + timeSlice * aggBlockedRate;
+    } else {
+      aggBlockedRate = 0.0f;
+    }
+    return aggBlockedRate;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3abd52bd/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
index 341a4bf..2cae887 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -148,6 +149,56 @@ public class TestRegionSplitPolicy {
     assertWithinJitter(maxSplitSize, policy.getSizeToCheck(0));
   }
 
+  @Test
+  public void testBusyRegionSplitPolicy() throws Exception {
+    conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
+        BusyRegionSplitPolicy.class.getName());
+    conf.setLong("hbase.busy.policy.minAge", 1000000L);
+    conf.setFloat("hbase.busy.policy.blockedRequests", 0.1f);
+
+    RegionServerServices rss  = Mockito.mock(RegionServerServices.class);
+    final List<Region> regions = new ArrayList<Region>();
+    Mockito.when(rss.getOnlineRegions(TABLENAME)).thenReturn(regions);
+    Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
+    Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L);
+    Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(0L);
+
+
+    BusyRegionSplitPolicy policy =
+        (BusyRegionSplitPolicy)RegionSplitPolicy.create(mockRegion, conf);
+
+    Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(10L);
+    Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(10L);
+    // Not enough time since region came online
+    assertFalse(policy.shouldSplit());
+
+
+    // Reset min age for split to zero
+    conf.setLong("hbase.busy.policy.minAge", 0L);
+    // Aggregate over 500 ms periods
+    conf.setLong("hbase.busy.policy.aggWindow", 500L);
+    policy =
+        (BusyRegionSplitPolicy)RegionSplitPolicy.create(mockRegion, conf);
+    long start = EnvironmentEdgeManager.currentTime();
+    Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(10L);
+    Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(20L);
+    Thread.sleep(300);
+    assertFalse(policy.shouldSplit());
+    Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(12L);
+    Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(30L);
+    Thread.sleep(2);
+    // Enough blocked requests since last time, but aggregate blocked request
+    // rate over last 500 ms is still low, because major portion of the window is constituted
+    // by the previous zero blocked request period which lasted at least 300 ms off last 500 ms.
+    if (EnvironmentEdgeManager.currentTime() - start < 500) {
+      assertFalse(policy.shouldSplit());
+    }
+    Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(14L);
+    Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(40L);
+    Thread.sleep(200);
+    assertTrue(policy.shouldSplit());
+  }
+
   private void assertWithinJitter(long maxSplitSize, long sizeToCheck) {
     assertTrue("Size greater than lower bound of jitter",
         (long)(maxSplitSize * 0.75) <= sizeToCheck);


[28/30] hbase git commit: HBASE-16045 endtime argument for VerifyReplication was incorrectly specified in usage

Posted by sy...@apache.org.
HBASE-16045 endtime argument for VerifyReplication was incorrectly specified in usage


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

Branch: refs/heads/hbase-12439
Commit: d8902ba0e68ec7bc38a8aa8d212353c380e5d378
Parents: 62a4a2c
Author: tedyu <yu...@gmail.com>
Authored: Thu Jun 16 15:50:29 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Jun 16 15:50:29 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/replication/VerifyReplication.java  | 6 +++++-
 src/main/asciidoc/_chapters/ops_mgt.adoc                       | 2 +-
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d8902ba0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 655c71a..f3175fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -400,6 +400,10 @@ public class VerifyReplication extends Configured implements Tool {
           continue;
         }
 
+        if (cmd.startsWith("--")) {
+          printUsage("Invalid argument '" + cmd + "'");
+        }
+
         if (i == args.length-2) {
           peerId = cmd;
         }
@@ -435,7 +439,7 @@ public class VerifyReplication extends Configured implements Tool {
       System.err.println("ERROR: " + errorMsg);
     }
     System.err.println("Usage: verifyrep [--starttime=X]" +
-        " [--stoptime=Y] [--families=A] [--row-prefixes=B] <peerid> <tablename>");
+        " [--endtime=Y] [--families=A] [--row-prefixes=B] <peerid> <tablename>");
     System.err.println();
     System.err.println("Options:");
     System.err.println(" starttime    beginning of the time range");

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8902ba0/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index e7ca940..590854e 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -1384,7 +1384,7 @@ The `VerifyReplication` MapReduce job, which is included in HBase, performs a sy
 +
 [source,bash]
 ----
-$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` "${HADOOP_HOME}/bin/hadoop" jar "${HBASE_HOME}/hbase-server-VERSION.jar" verifyrep --starttime=<timestamp> --stoptime=<timestamp> --families=<myFam> <ID> <tableName>
+$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` "${HADOOP_HOME}/bin/hadoop" jar "${HBASE_HOME}/hbase-server-VERSION.jar" verifyrep --starttime=<timestamp> --endtime=<timestamp> --families=<myFam> <ID> <tableName>
 ----
 +
 The `VerifyReplication` command prints out `GOODROWS` and `BADROWS` counters to indicate rows that did and did not replicate correctly.


[07/30] hbase git commit: HBASE-15525 OutOfMemory could occur when using BoundedByteBufferPool during RPC bursts.

Posted by sy...@apache.org.
HBASE-15525 OutOfMemory could occur when using BoundedByteBufferPool during RPC bursts.


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

Branch: refs/heads/hbase-12439
Commit: 17bcf14fea2637fe0e5ca23bb0008c1cca208c98
Parents: e486d27
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Jun 14 19:03:54 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Tue Jun 14 19:03:54 2016 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/IPCUtil.java    | 107 ++++++------
 .../hbase/io/ByteBufferListOutputStream.java    | 173 +++++++++++++++++++
 .../hadoop/hbase/io/ByteBufferOutputStream.java |  46 ++---
 .../apache/hadoop/hbase/io/ByteBufferPool.java  | 154 +++++++++++++++++
 .../io/TestByteBufferListOutputStream.java      |  77 +++++++++
 .../hadoop/hbase/io/TestByteBufferPool.java     |  60 +++++++
 .../apache/hadoop/hbase/ipc/BufferChain.java    |   7 +
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  99 +++++++----
 8 files changed, 609 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index 74466b5..74f934c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -34,10 +34,10 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
-import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.io.ByteBufferPool;
+import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.compress.CodecPool;
@@ -90,30 +90,7 @@ public class IPCUtil {
    */
   @SuppressWarnings("resource")
   public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
-    final CellScanner cellScanner)
-  throws IOException {
-    return buildCellBlock(codec, compressor, cellScanner, null);
-  }
-
-  /**
-   * Puts CellScanner Cells into a cell block using passed in <code>codec</code> and/or
-   * <code>compressor</code>.
-   * @param codec to use for encoding
-   * @param compressor to use for encoding
-   * @param cellScanner to encode
-   * @param pool Pool of ByteBuffers to make use of. Can be null and then we'll allocate
-   *   our own ByteBuffer.
-   * @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using
-   *   passed in <code>codec</code> and/or <code>compressor</code>; the returned buffer has been
-   *   flipped and is ready for reading.  Use limit to find total size. If <code>pool</code> was not
-   *   null, then this returned ByteBuffer came from there and should be returned to the pool when
-   *   done.
-   * @throws IOException if encoding the cells fail
-   */
-  @SuppressWarnings("resource")
-  public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
-    final CellScanner cellScanner, final BoundedByteBufferPool pool)
-  throws IOException {
+      final CellScanner cellScanner) throws IOException {
     if (cellScanner == null) {
       return null;
     }
@@ -121,25 +98,25 @@ public class IPCUtil {
       throw new CellScannerButNoCodecException();
     }
     int bufferSize = this.cellBlockBuildingInitialBufferSize;
-    ByteBufferOutputStream baos;
-    if (pool != null) {
-      ByteBuffer bb = pool.getBuffer();
-      bufferSize = bb.capacity();
-      baos = new ByteBufferOutputStream(bb);
-    } else {
-      // Then we need to make our own to return.
-      if (cellScanner instanceof HeapSize) {
-        long longSize = ((HeapSize)cellScanner).heapSize();
-        // Just make sure we don't have a size bigger than an int.
-        if (longSize > Integer.MAX_VALUE) {
-          throw new IOException("Size " + longSize + " > " + Integer.MAX_VALUE);
-        }
-        bufferSize = ClassSize.align((int)longSize);
+    ByteBufferOutputStream baos = new ByteBufferOutputStream(bufferSize);
+    encodeCellsTo(baos, cellScanner, codec, compressor);
+    if (LOG.isTraceEnabled()) {
+      if (bufferSize < baos.size()) {
+        LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + baos.size()
+            + "; up hbase.ipc.cellblock.building.initial.buffersize?");
       }
-      baos = new ByteBufferOutputStream(bufferSize);
     }
+    ByteBuffer bb = baos.getByteBuffer();
+    // If no cells, don't mess around. Just return null (could be a bunch of existence checking
+    // gets or something -- stuff that does not return a cell).
+    if (!bb.hasRemaining()) return null;
+    return bb;
+  }
+
+  private void encodeCellsTo(ByteBufferOutputStream bbos, CellScanner cellScanner, Codec codec,
+      CompressionCodec compressor) throws IOException {
+    OutputStream os = bbos;
     Compressor poolCompressor = null;
-    OutputStream os = baos;
     try  {
       if (compressor != null) {
         if (compressor instanceof Configurable) {
@@ -149,33 +126,51 @@ public class IPCUtil {
         os = compressor.createOutputStream(os, poolCompressor);
       }
       Codec.Encoder encoder = codec.getEncoder(os);
-      int count = 0;
       while (cellScanner.advance()) {
         encoder.write(cellScanner.current());
-        count++;
       }
       encoder.flush();
-      // If no cells, don't mess around.  Just return null (could be a bunch of existence checking
-      // gets or something -- stuff that does not return a cell).
-      if (count == 0) {
-        return null;
-      }
     } catch (BufferOverflowException e) {
       throw new DoNotRetryIOException(e);
     } finally {
       os.close();
-
       if (poolCompressor != null) {
         CodecPool.returnCompressor(poolCompressor);
       }
     }
-    if (LOG.isTraceEnabled()) {
-      if (bufferSize < baos.size()) {
-        LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + baos.size() +
-          "; up hbase.ipc.cellblock.building.initial.buffersize?");
-      }
+  }
+
+  /**
+   * Puts CellScanner Cells into a cell block using passed in <code>codec</code> and/or
+   * <code>compressor</code>.
+   * @param codec to use for encoding
+   * @param compressor to use for encoding
+   * @param cellScanner to encode
+   * @param pool Pool of ByteBuffers to make use of.
+   * @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using
+   *   passed in <code>codec</code> and/or <code>compressor</code>; the returned buffer has been
+   *   flipped and is ready for reading.  Use limit to find total size. If <code>pool</code> was not
+   *   null, then this returned ByteBuffer came from there and should be returned to the pool when
+   *   done.
+   * @throws IOException if encoding the cells fail
+   */
+  @SuppressWarnings("resource")
+  public ByteBufferListOutputStream buildCellBlockStream(Codec codec, CompressionCodec compressor,
+      CellScanner cellScanner, ByteBufferPool pool) throws IOException {
+    if (cellScanner == null) {
+      return null;
+    }
+    if (codec == null) {
+      throw new CellScannerButNoCodecException();
+    }
+    assert pool != null;
+    ByteBufferListOutputStream bbos = new ByteBufferListOutputStream(pool);
+    encodeCellsTo(bbos, cellScanner, codec, compressor);
+    if (bbos.size() == 0) {
+      bbos.releaseResources();
+      return null;
     }
-    return baos.getByteBuffer();
+    return bbos;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferListOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferListOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferListOutputStream.java
new file mode 100644
index 0000000..b4c00c6
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferListOutputStream.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * An OutputStream which writes data into ByteBuffers. It will try to get ByteBuffer, as and when
+ * needed, from the passed pool. When pool is not giving a ByteBuffer it will create one on heap.
+ * Make sure to call {@link #releaseResources()} method once the Stream usage is over and
+ * data is transferred to the wanted destination.
+ * Not thread safe!
+ */
+@InterfaceAudience.Private
+public class ByteBufferListOutputStream extends ByteBufferOutputStream {
+  private static final Log LOG = LogFactory.getLog(ByteBufferListOutputStream.class);
+
+  private ByteBufferPool pool;
+  // Keep track of the BBs where bytes written to. We will first try to get a BB from the pool. If
+  // it is not available will make a new one our own and keep writing to that. We keep track of all
+  // the BBs that we got from pool, separately so that on closeAndPutbackBuffers, we can make sure
+  // to return back all of them to pool
+  protected List<ByteBuffer> allBufs = new ArrayList<ByteBuffer>();
+  protected List<ByteBuffer> bufsFromPool = new ArrayList<ByteBuffer>();
+
+  private boolean lastBufFlipped = false;// Indicate whether the curBuf/lastBuf is flipped already
+
+  public ByteBufferListOutputStream(ByteBufferPool pool) {
+    this.pool = pool;
+    allocateNewBuffer();
+  }
+
+  private void allocateNewBuffer() {
+    if (this.curBuf != null) {
+      this.curBuf.flip();// On the current buf set limit = pos and pos = 0.
+    }
+    // Get an initial BB to work with from the pool
+    this.curBuf = this.pool.getBuffer();
+    if (this.curBuf == null) {
+      // No free BB at this moment. Make a new one. The pool returns off heap BBs. Don't make off
+      // heap BB on demand. It is difficult to account for all such and so proper sizing of Max
+      // direct heap size. See HBASE-15525 also for more details.
+      // Make BB with same size of pool's buffer size.
+      this.curBuf = ByteBuffer.allocate(this.pool.getBufferSize());
+    } else {
+      this.bufsFromPool.add(this.curBuf);
+    }
+    this.allBufs.add(this.curBuf);
+  }
+
+  @Override
+  public int size() {
+    int s = 0;
+    for (int i = 0; i < this.allBufs.size() - 1; i++) {
+      s += this.allBufs.get(i).remaining();
+    }
+    // On the last BB, it might not be flipped yet if getByteBuffers is not yet called
+    if (this.lastBufFlipped) {
+      s += this.curBuf.remaining();
+    } else {
+      s += this.curBuf.position();
+    }
+    return s;
+  }
+
+  @Override
+  public ByteBuffer getByteBuffer() {
+    throw new UnsupportedOperationException("This stream is not backed by a single ByteBuffer");
+  }
+
+  @Override
+  protected void checkSizeAndGrow(int extra) {
+    long capacityNeeded = curBuf.position() + (long) extra;
+    if (capacityNeeded > curBuf.limit()) {
+      allocateNewBuffer();
+    }
+  }
+
+  @Override
+  public void writeTo(OutputStream out) throws IOException {
+    // No usage of this API in code. Just making it as an Unsupported operation as of now
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Release the resources it uses (The ByteBuffers) which are obtained from pool. Call this only
+   * when all the data is fully used. And it must be called at the end of usage else we will leak
+   * ByteBuffers from pool.
+   */
+  public void releaseResources() {
+    try {
+      close();
+    } catch (IOException e) {
+      LOG.debug(e);
+    }
+    // Return back all the BBs to pool
+    if (this.bufsFromPool != null) {
+      for (int i = 0; i < this.bufsFromPool.size(); i++) {
+        this.pool.putbackBuffer(this.bufsFromPool.get(i));
+      }
+      this.bufsFromPool = null;
+    }
+    this.allBufs = null;
+    this.curBuf = null;
+  }
+
+  @Override
+  public byte[] toByteArray(int offset, int length) {
+    // No usage of this API in code. Just making it as an Unsupported operation as of now
+    throw new UnsupportedOperationException();
+  }
+
+  public List<ByteBuffer> getByteBuffers() {
+    if (!this.lastBufFlipped) {
+      this.lastBufFlipped = true;
+      // All the other BBs are already flipped while moving to the new BB.
+      curBuf.flip();
+    }
+    return this.allBufs;
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    int toWrite = 0;
+    while (len > 0) {
+      toWrite = Math.min(len, this.curBuf.remaining());
+      ByteBufferUtils.copyFromArrayToBuffer(this.curBuf, b, off, toWrite);
+      off += toWrite;
+      len -= toWrite;
+      if (len > 0) {
+        allocateNewBuffer();// The curBuf is over. Let us move to the next one
+      }
+    }
+  }
+
+  @Override
+  public void write(ByteBuffer b, int off, int len) throws IOException {
+    int toWrite = 0;
+    while (len > 0) {
+      toWrite = Math.min(len, this.curBuf.remaining());
+      ByteBufferUtils.copyFromBufferToBuffer(b, this.curBuf, off, toWrite);
+      off += toWrite;
+      len -= toWrite;
+      if (len > 0) {
+        allocateNewBuffer();// The curBuf is over. Let us move to the next one
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
index d4bda18..f77092d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
@@ -44,7 +44,11 @@ public class ByteBufferOutputStream extends OutputStream
   // http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/8-b132/java/util/ArrayList.java#221
   private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
 
-  protected ByteBuffer buf;
+  protected ByteBuffer curBuf = null;
+
+  ByteBufferOutputStream() {
+
+  }
 
   public ByteBufferOutputStream(int capacity) {
     this(capacity, false);
@@ -66,12 +70,12 @@ public class ByteBufferOutputStream extends OutputStream
    */
   public ByteBufferOutputStream(final ByteBuffer bb) {
     assert bb.order() == ByteOrder.BIG_ENDIAN;
-    this.buf = bb;
-    this.buf.clear();
+    this.curBuf = bb;
+    this.curBuf.clear();
   }
 
   public int size() {
-    return buf.position();
+    return curBuf.position();
   }
 
   private static ByteBuffer allocate(final int capacity, final boolean useDirectByteBuffer) {
@@ -86,25 +90,25 @@ public class ByteBufferOutputStream extends OutputStream
    * @return ByteBuffer
    */
   public ByteBuffer getByteBuffer() {
-    buf.flip();
-    return buf;
+    curBuf.flip();
+    return curBuf;
   }
 
-  private void checkSizeAndGrow(int extra) {
-    long capacityNeeded = buf.position() + (long) extra;
-    if (capacityNeeded > buf.limit()) {
+  protected void checkSizeAndGrow(int extra) {
+    long capacityNeeded = curBuf.position() + (long) extra;
+    if (capacityNeeded > curBuf.limit()) {
       // guarantee it's possible to fit
       if (capacityNeeded > MAX_ARRAY_SIZE) {
         throw new BufferOverflowException();
       }
       // double until hit the cap
-      long nextCapacity = Math.min(buf.capacity() * 2L, MAX_ARRAY_SIZE);
+      long nextCapacity = Math.min(curBuf.capacity() * 2L, MAX_ARRAY_SIZE);
       // but make sure there is enough if twice the existing capacity is still too small
       nextCapacity = Math.max(nextCapacity, capacityNeeded);
-      ByteBuffer newBuf = allocate((int) nextCapacity, buf.isDirect());
-      buf.flip();
-      ByteBufferUtils.copyFromBufferToBuffer(buf, newBuf);
-      buf = newBuf;
+      ByteBuffer newBuf = allocate((int) nextCapacity, curBuf.isDirect());
+      curBuf.flip();
+      ByteBufferUtils.copyFromBufferToBuffer(curBuf, newBuf);
+      curBuf = newBuf;
     }
   }
 
@@ -112,7 +116,7 @@ public class ByteBufferOutputStream extends OutputStream
   @Override
   public void write(int b) throws IOException {
     checkSizeAndGrow(Bytes.SIZEOF_BYTE);
-    buf.put((byte)b);
+    curBuf.put((byte)b);
   }
 
  /**
@@ -122,9 +126,9 @@ public class ByteBufferOutputStream extends OutputStream
   * @param      out   the output stream to which to write the data.
   * @exception  IOException  if an I/O error occurs.
   */
-  public synchronized void writeTo(OutputStream out) throws IOException {
+  public void writeTo(OutputStream out) throws IOException {
     WritableByteChannel channel = Channels.newChannel(out);
-    ByteBuffer bb = buf.duplicate();
+    ByteBuffer bb = curBuf.duplicate();
     bb.flip();
     channel.write(bb);
   }
@@ -137,12 +141,12 @@ public class ByteBufferOutputStream extends OutputStream
   @Override
   public void write(byte[] b, int off, int len) throws IOException {
     checkSizeAndGrow(len);
-    ByteBufferUtils.copyFromArrayToBuffer(buf, b, off, len);
+    ByteBufferUtils.copyFromArrayToBuffer(curBuf, b, off, len);
   }
 
   public void write(ByteBuffer b, int off, int len) throws IOException {
     checkSizeAndGrow(len);
-    ByteBufferUtils.copyFromBufferToBuffer(b, buf, off, len);
+    ByteBufferUtils.copyFromBufferToBuffer(b, curBuf, off, len);
   }
 
   /**
@@ -153,7 +157,7 @@ public class ByteBufferOutputStream extends OutputStream
    */
   public void writeInt(int i) throws IOException {
     checkSizeAndGrow(Bytes.SIZEOF_INT);
-    ByteBufferUtils.putInt(this.buf, i);
+    ByteBufferUtils.putInt(this.curBuf, i);
   }
 
   @Override
@@ -167,7 +171,7 @@ public class ByteBufferOutputStream extends OutputStream
   }
 
   public byte[] toByteArray(int offset, int length) {
-    ByteBuffer bb = buf.duplicate();
+    ByteBuffer bb = curBuf.duplicate();
     bb.flip();
 
     byte[] chunk = new byte[length];

http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferPool.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferPool.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferPool.java
new file mode 100644
index 0000000..e528f02
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferPool.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io;
+
+import java.nio.ByteBuffer;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Like Hadoops' ByteBufferPool only you do not specify desired size when getting a ByteBuffer. This
+ * pool keeps an upper bound on the count of ByteBuffers in the pool and a fixed size of ByteBuffer
+ * that it will create. When requested, if a free ByteBuffer is already present, it will return
+ * that. And when no free ByteBuffer available and we are below the max count, it will create a new
+ * one and return that.
+ *
+ * <p>
+ * Note: This pool returns off heap ByteBuffers by default. If on heap ByteBuffers to be pooled,
+ * pass 'directByteBuffer' as false while construction of the pool.
+ * <p>
+ * This class is thread safe.
+ *
+ * @see ByteBufferListOutputStream
+ */
+@InterfaceAudience.Private
+public class ByteBufferPool {
+  private static final Log LOG = LogFactory.getLog(ByteBufferPool.class);
+  // TODO better config names?
+  // hbase.ipc.server.reservoir.initial.max -> hbase.ipc.server.reservoir.max.buffer.count
+  // hbase.ipc.server.reservoir.initial.buffer.size -> hbase.ipc.server.reservoir.buffer.size
+  public static final String MAX_POOL_SIZE_KEY = "hbase.ipc.server.reservoir.initial.max";
+  public static final String BUFFER_SIZE_KEY = "hbase.ipc.server.reservoir.initial.buffer.size";
+  public static final int DEFAULT_BUFFER_SIZE = 64 * 1024;// 64 KB. Making it same as the chunk size
+                                                          // what we will write/read to/from the
+                                                          // socket channel.
+  private final Queue<ByteBuffer> buffers = new ConcurrentLinkedQueue<ByteBuffer>();
+
+  private final int bufferSize;
+  private final int maxPoolSize;
+  private AtomicInteger count; // Count of the BBs created already for this pool.
+  private final boolean directByteBuffer; //Whether this pool should return DirectByteBuffers
+  private boolean maxPoolSizeInfoLevelLogged = false;
+
+  /**
+   * @param bufferSize Size of each buffer created by this pool.
+   * @param maxPoolSize Max number of buffers to keep in this pool.
+   */
+  public ByteBufferPool(int bufferSize, int maxPoolSize) {
+    this(bufferSize, maxPoolSize, true);
+  }
+
+  /**
+   * @param bufferSize Size of each buffer created by this pool.
+   * @param maxPoolSize Max number of buffers to keep in this pool.
+   * @param directByteBuffer Whether to create direct ByteBuffer or on heap ByteBuffer.
+   */
+  public ByteBufferPool(int bufferSize, int maxPoolSize, boolean directByteBuffer) {
+    this.bufferSize = bufferSize;
+    this.maxPoolSize = maxPoolSize;
+    this.directByteBuffer = directByteBuffer;
+    // TODO can add initialPoolSize config also and make those many BBs ready for use.
+    LOG.info("Created ByteBufferPool with bufferSize : " + bufferSize + " and maxPoolSize : "
+        + maxPoolSize);
+    this.count = new AtomicInteger(0);
+  }
+
+  /**
+   * @return One free ByteBuffer from the pool. If no free ByteBuffer and we have not reached the
+   *         maximum pool size, it will create a new one and return. In case of max pool size also
+   *         reached, will return null. When pool returned a ByteBuffer, make sure to return it back
+   *         to pool after use.
+   * @see #putbackBuffer(ByteBuffer)
+   */
+  public ByteBuffer getBuffer() {
+    ByteBuffer bb = buffers.poll();
+    if (bb != null) {
+      // Clear sets limit == capacity. Position == 0.
+      bb.clear();
+      return bb;
+    }
+    while (true) {
+      int c = this.count.intValue();
+      if (c >= this.maxPoolSize) {
+        if (maxPoolSizeInfoLevelLogged) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Pool already reached its max capacity : " + this.maxPoolSize
+                + " and no free buffers now. Consider increasing the value for '"
+                + MAX_POOL_SIZE_KEY + "' ?");
+          }
+        } else {
+          LOG.info("Pool already reached its max capacity : " + this.maxPoolSize
+              + " and no free buffers now. Consider increasing the value for '" + MAX_POOL_SIZE_KEY
+              + "' ?");
+          maxPoolSizeInfoLevelLogged = true;
+        }
+        return null;
+      }
+      if (!this.count.compareAndSet(c, c + 1)) {
+        continue;
+      }
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Creating a new offheap ByteBuffer of size: " + this.bufferSize);
+      }
+      return this.directByteBuffer ? ByteBuffer.allocateDirect(this.bufferSize)
+          : ByteBuffer.allocate(this.bufferSize);
+    }
+  }
+
+  /**
+   * Return back a ByteBuffer after its use. Do not try to return put back a ByteBuffer, not
+   * obtained from this pool.
+   * @param buf ByteBuffer to return.
+   */
+  public void putbackBuffer(ByteBuffer buf) {
+    if (buf.capacity() != this.bufferSize || (this.directByteBuffer ^ buf.isDirect())) {
+      LOG.warn("Trying to put a buffer, not created by this pool! Will be just ignored");
+      return;
+    }
+    buffers.offer(buf);
+  }
+
+  int getBufferSize() {
+    return this.bufferSize;
+  }
+
+  /**
+   * @return Number of free buffers
+   */
+  @VisibleForTesting
+  int getQueueSize() {
+    return buffers.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java
new file mode 100644
index 0000000..e1d1e04
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferListOutputStream.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ IOTests.class, SmallTests.class })
+public class TestByteBufferListOutputStream {
+
+  @Test
+  public void testWrites() throws Exception {
+    ByteBufferPool pool = new ByteBufferPool(10, 3);
+    ByteBufferListOutputStream bbos = new ByteBufferListOutputStream(pool);
+    bbos.write(2);// Write a byte
+    bbos.writeInt(100);// Write an int
+    byte[] b = Bytes.toBytes("row123");// 6 bytes
+    bbos.write(b);
+    // Just use the 3rd BB from pool so that pabos, on request, wont get one
+    ByteBuffer bb1 = pool.getBuffer();
+    ByteBuffer bb = ByteBuffer.wrap(Bytes.toBytes("row123_cf1_q1"));// 13 bytes
+    bbos.write(bb, 0, bb.capacity());
+    pool.putbackBuffer(bb1);
+    bbos.writeInt(123);
+    bbos.writeInt(124);
+    assertEquals(0, pool.getQueueSize());
+    List<ByteBuffer> allBufs = bbos.getByteBuffers();
+    assertEquals(4, allBufs.size());
+    assertEquals(3, bbos.bufsFromPool.size());
+    ByteBuffer b1 = allBufs.get(0);
+    assertEquals(10, b1.remaining());
+    assertEquals(2, b1.get());
+    assertEquals(100, b1.getInt());
+    byte[] bActual = new byte[b.length];
+    b1.get(bActual, 0, 5);//5 bytes in 1st BB
+    ByteBuffer b2 = allBufs.get(1);
+    assertEquals(10, b2.remaining());
+    b2.get(bActual, 5, 1);// Remaining 1 byte in 2nd BB
+    assertTrue(Bytes.equals(b, bActual));
+    bActual = new byte[bb.capacity()];
+    b2.get(bActual, 0, 9);
+    ByteBuffer b3 = allBufs.get(2);
+    assertEquals(8, b3.remaining());
+    b3.get(bActual, 9, 4);
+    assertTrue(ByteBufferUtils.equals(bb, 0, bb.capacity(), bActual, 0, bActual.length));
+    assertEquals(123, b3.getInt());
+    ByteBuffer b4 = allBufs.get(3);
+    assertEquals(4, b4.remaining());
+    assertEquals(124, b4.getInt());
+    bbos.releaseResources();
+    assertEquals(3, pool.getQueueSize());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java
new file mode 100644
index 0000000..64a4103
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferPool.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import static org.junit.Assert.assertEquals;
+@Category({ IOTests.class, SmallTests.class })
+public class TestByteBufferPool {
+
+  @Test
+  public void testOffheapBBPool() throws Exception {
+    boolean directByteBuffer = true;
+    testBBPool(10, 100, directByteBuffer);
+  }
+
+  @Test
+  public void testOnheapBBPool() throws Exception {
+    boolean directByteBuffer = false;
+    testBBPool(10, 100, directByteBuffer);
+  }
+
+  private void testBBPool(int maxPoolSize, int bufferSize, boolean directByteBuffer) {
+    ByteBufferPool pool = new ByteBufferPool(bufferSize, maxPoolSize, directByteBuffer);
+    for (int i = 0; i < maxPoolSize; i++) {
+      ByteBuffer buffer = pool.getBuffer();
+      assertEquals(0, buffer.position());
+      assertEquals(bufferSize, buffer.limit());
+      assertEquals(directByteBuffer, buffer.isDirect());
+    }
+    assertEquals(0, pool.getQueueSize());
+    ByteBuffer bb = directByteBuffer ? ByteBuffer.allocate(bufferSize)
+        : ByteBuffer.allocateDirect(bufferSize);
+    pool.putbackBuffer(bb);
+    assertEquals(0, pool.getQueueSize());
+    bb = directByteBuffer ? ByteBuffer.allocateDirect(bufferSize + 1)
+        : ByteBuffer.allocate(bufferSize + 1);
+    pool.putbackBuffer(bb);
+    assertEquals(0, pool.getQueueSize());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BufferChain.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BufferChain.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BufferChain.java
index babd2f8..39efa40 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BufferChain.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BufferChain.java
@@ -46,6 +46,13 @@ class BufferChain {
     this.buffers = bbs.toArray(new ByteBuffer[bbs.size()]);
   }
 
+  BufferChain(List<ByteBuffer> buffers) {
+    for (ByteBuffer b : buffers) {
+      this.remaining += b.remaining();
+    }
+    this.buffers = buffers.toArray(new ByteBuffer[buffers.size()]);
+  }
+
   /**
    * Expensive.  Makes a new buffer to hold a copy of what is in contained ByteBuffers.  This
    * call drains this instance; it cannot be used subsequent to the call.

http://git-wip-us.apache.org/repos/asf/hbase/blob/17bcf14f/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 1087c42..c7f5a10 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -85,9 +85,10 @@ import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
-import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
+import org.apache.hadoop.hbase.io.ByteBufferPool;
+import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -289,7 +290,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
 
   private UserProvider userProvider;
 
-  private final BoundedByteBufferPool reservoir;
+  private final ByteBufferPool reservoir;
 
   private volatile boolean allowFallbackToSimpleAuth;
 
@@ -320,7 +321,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     protected long size;                          // size of current call
     protected boolean isError;
     protected TraceInfo tinfo;
-    private ByteBuffer cellBlock = null;
+    private ByteBufferListOutputStream cellBlockStream = null;
 
     private User user;
     private InetAddress remoteAddress;
@@ -362,10 +363,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
         justification="Presume the lock on processing request held by caller is protection enough")
     void done() {
-      if (this.cellBlock != null && reservoir != null) {
-        // Return buffer to reservoir now we are done with it.
-        reservoir.putBuffer(this.cellBlock);
-        this.cellBlock = null;
+      if (this.cellBlockStream != null) {
+        this.cellBlockStream.releaseResources();// This will return back the BBs which we
+                                                // got from pool.
+        this.cellBlockStream = null;
       }
       this.connection.decRpcCount();  // Say that we're done with this call.
     }
@@ -425,38 +426,43 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         // Call id.
         headerBuilder.setCallId(this.id);
         if (t != null) {
-          ExceptionResponse.Builder exceptionBuilder = ExceptionResponse.newBuilder();
-          exceptionBuilder.setExceptionClassName(t.getClass().getName());
-          exceptionBuilder.setStackTrace(errorMsg);
-          exceptionBuilder.setDoNotRetry(t instanceof DoNotRetryIOException);
-          if (t instanceof RegionMovedException) {
-            // Special casing for this exception.  This is only one carrying a payload.
-            // Do this instead of build a generic system for allowing exceptions carry
-            // any kind of payload.
-            RegionMovedException rme = (RegionMovedException)t;
-            exceptionBuilder.setHostname(rme.getHostname());
-            exceptionBuilder.setPort(rme.getPort());
-          }
-          // Set the exception as the result of the method invocation.
-          headerBuilder.setException(exceptionBuilder.build());
+          setExceptionResponse(t, errorMsg, headerBuilder);
         }
         // Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the
         // reservoir when finished. This is hacky and the hack is not contained but benefits are
         // high when we can avoid a big buffer allocation on each rpc.
-        this.cellBlock = ipcUtil.buildCellBlock(this.connection.codec,
-          this.connection.compressionCodec, cells, reservoir);
-        if (this.cellBlock != null) {
+        List<ByteBuffer> cellBlock = null;
+        int cellBlockSize = 0;
+        if (reservoir != null) {
+          this.cellBlockStream = ipcUtil.buildCellBlockStream(this.connection.codec,
+              this.connection.compressionCodec, cells, reservoir);
+          if (this.cellBlockStream != null) {
+            cellBlock = this.cellBlockStream.getByteBuffers();
+            cellBlockSize = this.cellBlockStream.size();
+          }
+        } else {
+          ByteBuffer b = ipcUtil.buildCellBlock(this.connection.codec,
+              this.connection.compressionCodec, cells);
+          if (b != null) {
+            cellBlockSize = b.remaining();
+            cellBlock = new ArrayList<ByteBuffer>(1);
+            cellBlock.add(b);
+          }
+        }
+
+        if (cellBlockSize > 0) {
           CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
           // Presumes the cellBlock bytebuffer has been flipped so limit has total size in it.
-          cellBlockBuilder.setLength(this.cellBlock.limit());
+          cellBlockBuilder.setLength(cellBlockSize);
           headerBuilder.setCellBlockMeta(cellBlockBuilder.build());
         }
         Message header = headerBuilder.build();
-
-        byte[] b = createHeaderAndMessageBytes(result, header);
-
-        bc = new BufferChain(ByteBuffer.wrap(b), this.cellBlock);
-
+        byte[] b = createHeaderAndMessageBytes(result, header, cellBlockSize);
+        List<ByteBuffer> responseBufs = new ArrayList<ByteBuffer>(
+            (cellBlock == null ? 1 : cellBlock.size()) + 1);
+        responseBufs.add(ByteBuffer.wrap(b));
+        if (cellBlock != null) responseBufs.addAll(cellBlock);
+        bc = new BufferChain(responseBufs);
         if (connection.useWrap) {
           bc = wrapWithSasl(bc);
         }
@@ -476,7 +482,25 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       }
     }
 
-    private byte[] createHeaderAndMessageBytes(Message result, Message header)
+    private void setExceptionResponse(Throwable t, String errorMsg,
+        ResponseHeader.Builder headerBuilder) {
+      ExceptionResponse.Builder exceptionBuilder = ExceptionResponse.newBuilder();
+      exceptionBuilder.setExceptionClassName(t.getClass().getName());
+      exceptionBuilder.setStackTrace(errorMsg);
+      exceptionBuilder.setDoNotRetry(t instanceof DoNotRetryIOException);
+      if (t instanceof RegionMovedException) {
+        // Special casing for this exception.  This is only one carrying a payload.
+        // Do this instead of build a generic system for allowing exceptions carry
+        // any kind of payload.
+        RegionMovedException rme = (RegionMovedException)t;
+        exceptionBuilder.setHostname(rme.getHostname());
+        exceptionBuilder.setPort(rme.getPort());
+      }
+      // Set the exception as the result of the method invocation.
+      headerBuilder.setException(exceptionBuilder.build());
+    }
+
+    private byte[] createHeaderAndMessageBytes(Message result, Message header, int cellBlockSize)
         throws IOException {
       // Organize the response as a set of bytebuffers rather than collect it all together inside
       // one big byte array; save on allocations.
@@ -493,7 +517,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       // calculate the total size
       int totalSize = headerSerializedSize + headerVintSize
           + (resultSerializedSize + resultVintSize)
-          + (this.cellBlock == null ? 0 : this.cellBlock.limit());
+          + cellBlockSize;
       // The byte[] should also hold the totalSize of the header, message and the cellblock
       byte[] b = new byte[headerSerializedSize + headerVintSize + resultSerializedSize
           + resultVintSize + Bytes.SIZEOF_INT];
@@ -1084,6 +1108,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       } finally {
         if (error) {
           LOG.debug(getName() + call.toShortString() + ": output error -- closing");
+          // We will be closing this connection itself. Mark this call as done so that all the
+          // buffer(s) it got from pool can get released
+          call.done();
           closeConnection(call.connection);
         }
       }
@@ -1998,11 +2025,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       RpcScheduler scheduler)
       throws IOException {
     if (conf.getBoolean("hbase.ipc.server.reservoir.enabled", true)) {
-      this.reservoir = new BoundedByteBufferPool(
-          conf.getInt("hbase.ipc.server.reservoir.max.buffer.size", 1024 * 1024),
-          conf.getInt("hbase.ipc.server.reservoir.initial.buffer.size", 16 * 1024),
-          // Make the max twice the number of handlers to be safe.
-          conf.getInt("hbase.ipc.server.reservoir.initial.max",
+      this.reservoir = new ByteBufferPool(
+          conf.getInt(ByteBufferPool.BUFFER_SIZE_KEY, ByteBufferPool.DEFAULT_BUFFER_SIZE),
+          conf.getInt(ByteBufferPool.MAX_POOL_SIZE_KEY,
               conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
                   HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT) * 2));
     } else {


[09/30] hbase git commit: HBASE-15344 add 1.3 to prereq tables in ref guide

Posted by sy...@apache.org.
HBASE-15344 add 1.3 to prereq tables in ref guide


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

Branch: refs/heads/hbase-12439
Commit: 1621257e7abbe6fe924373ea09528736aa754b78
Parents: df24004
Author: Mikhail Antonov <an...@apache.org>
Authored: Fri Jun 10 16:53:37 2016 -0700
Committer: Mikhail Antonov <an...@apache.org>
Committed: Tue Jun 14 14:59:31 2016 -0700

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/configuration.adoc | 36 ++++++++++++---------
 1 file changed, 21 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1621257e/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index 4702bcb..82db83d 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -100,6 +100,12 @@ This section lists required services and some required system configuration.
 |JDK 7
 |JDK 8
 
+|1.3
+|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
+|yes
+|yes
+
+
 |1.2
 |link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
 |yes
@@ -214,22 +220,22 @@ Use the following legend to interpret this table:
 * "X" = not supported
 * "NT" = Not tested
 
-[cols="1,1,1,1,1,1", options="header"]
+[cols="1,1,1,1,1,1,1", options="header"]
 |===
-| | HBase-0.94.x | HBase-0.98.x (Support for Hadoop 1.1+ is deprecated.) | HBase-1.0.x (Hadoop 1.x is NOT supported) | HBase-1.1.x | HBase-1.2.x
-|Hadoop-1.0.x  | X | X | X | X | X
-|Hadoop-1.1.x | S | NT | X | X | X
-|Hadoop-0.23.x | S | X | X | X | X
-|Hadoop-2.0.x-alpha | NT | X | X | X | X
-|Hadoop-2.1.0-beta | NT | X | X | X | X
-|Hadoop-2.2.0 | NT | S | NT | NT | X 
-|Hadoop-2.3.x | NT | S | NT | NT | X 
-|Hadoop-2.4.x | NT | S | S | S | S
-|Hadoop-2.5.x | NT | S | S | S | S
-|Hadoop-2.6.0 | X | X | X | X | X
-|Hadoop-2.6.1+ | NT | NT | NT | NT | S
-|Hadoop-2.7.0 | X | X | X | X | X
-|Hadoop-2.7.1+ | NT | NT | NT | NT | S
+| | HBase-0.94.x | HBase-0.98.x (Support for Hadoop 1.1+ is deprecated.) | HBase-1.0.x (Hadoop 1.x is NOT supported) | HBase-1.1.x | HBase-1.2.x | HBase-1.3.x
+|Hadoop-1.0.x  | X | X | X | X | X | X
+|Hadoop-1.1.x | S | NT | X | X | X | X
+|Hadoop-0.23.x | S | X | X | X | X | X
+|Hadoop-2.0.x-alpha | NT | X | X | X | X | X
+|Hadoop-2.1.0-beta | NT | X | X | X | X | X
+|Hadoop-2.2.0 | NT | S | NT | NT | X  | X
+|Hadoop-2.3.x | NT | S | NT | NT | X  | X
+|Hadoop-2.4.x | NT | S | S | S | S | S
+|Hadoop-2.5.x | NT | S | S | S | S | S
+|Hadoop-2.6.0 | X | X | X | X | X | X
+|Hadoop-2.6.1+ | NT | NT | NT | NT | S | S
+|Hadoop-2.7.0 | X | X | X | X | X | X
+|Hadoop-2.7.1+ | NT | NT | NT | NT | S | S
 |===
 
 .Hadoop 2.6.x


[10/30] hbase git commit: HBASE-16021 graceful_stop.sh: Wrap variables in double quote to avoid "[: too many arguments" error (Samir Ahmic)

Posted by sy...@apache.org.
HBASE-16021 graceful_stop.sh: Wrap variables in double quote to avoid "[: too many arguments" error (Samir Ahmic)


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

Branch: refs/heads/hbase-12439
Commit: b69c77ac1a5f67ced25952f9cf85fac768427880
Parents: 1621257
Author: tedyu <yu...@gmail.com>
Authored: Tue Jun 14 15:36:11 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jun 14 15:36:11 2016 -0700

----------------------------------------------------------------------
 bin/graceful_stop.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b69c77ac/bin/graceful_stop.sh
----------------------------------------------------------------------
diff --git a/bin/graceful_stop.sh b/bin/graceful_stop.sh
index 510411e..8d08cc9 100755
--- a/bin/graceful_stop.sh
+++ b/bin/graceful_stop.sh
@@ -101,7 +101,7 @@ if [ "$localhostname" == "$hostname" ]; then
   local=true
 fi
 
-if [ $nob == "true"  ]; then
+if [ "$nob" == "true"  ]; then
   log "[ $0 ] skipping disabling balancer -nob argument is used"
   HBASE_BALANCER_STATE=false
 else
@@ -175,7 +175,7 @@ if [ "$restart" != "" ]; then
 fi
 
 # Restore balancer state
-if [ $HBASE_BALANCER_STATE != "false" ] && [ $nob != "true"  ]; then
+if [ "$HBASE_BALANCER_STATE" != "false" ] && [ "$nob" != "true"  ]; then
   log "Restoring balancer state to $HBASE_BALANCER_STATE"
   echo "balance_switch $HBASE_BALANCER_STATE" | "$bin"/hbase --config ${HBASE_CONF_DIR} shell &> /dev/null
 else


[22/30] hbase git commit: HBASE-16038 Ignore JVM crashes or machine shutdown failures in report-flakies.

Posted by sy...@apache.org.
HBASE-16038 Ignore JVM crashes or machine shutdown failures in report-flakies.

Change-Id: If49acd704e827b289c75f449a6180038b297d647


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

Branch: refs/heads/hbase-12439
Commit: f283ff01aa52a810e81553dfb67edd5d2fab5ddf
Parents: 1bad166
Author: Apekshit Sharma <ap...@apache.org>
Authored: Wed Jun 15 15:39:49 2016 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Wed Jun 15 16:01:49 2016 -0700

----------------------------------------------------------------------
 dev-support/findHangingTests.py | 18 +++++++++++++++++-
 dev-support/report-flakies.py   |  6 +++++-
 2 files changed, 22 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f283ff01/dev-support/findHangingTests.py
----------------------------------------------------------------------
diff --git a/dev-support/findHangingTests.py b/dev-support/findHangingTests.py
index 28f4895..c906125 100755
--- a/dev-support/findHangingTests.py
+++ b/dev-support/findHangingTests.py
@@ -23,6 +23,14 @@ import re
 import requests
 import sys
 
+
+# If any of these strings appear in the console output, it's a build one should probably ignore
+# for analyzing failed/hanging tests.
+BAD_RUN_STRINGS = [
+    "Slave went offline during the build",  # Machine went down, can't do anything about it.
+    "The forked VM terminated without properly saying goodbye",  # JVM crashed.
+]
+
 # Returns [[all tests], [failed tests], [timeout tests], [hanging tests]]
 # Definitions:
 # All tests: All testcases which were run.
@@ -66,6 +74,10 @@ def get_bad_tests(console_url):
         if result3:
             test_case = result3.group(1)
             timeout_tests.add(test_case)
+        for bad_string in BAD_RUN_STRINGS:
+            if re.match(".*" + bad_string + ".*", line):
+                print "Bad string found in build:\n > {}".format(line)
+                return
     print "Result > total tests: {:4}   failed : {:4}  timedout : {:4}  hanging : {:4}".format(
           len(all_tests), len(failed_tests), len(timeout_tests), len(hanging_tests))
     return [all_tests, failed_tests, timeout_tests, hanging_tests]
@@ -76,7 +88,11 @@ if __name__ == "__main__":
         sys.exit(1)
 
     print "Fetching {}".format(sys.argv[1])
-    [all_tests, failed_tests, timedout_tests, hanging_tests] = get_bad_tests(sys.argv[1])
+    result = get_bad_tests(sys.argv[1])
+    if not result:
+        sys.exit(1)
+    [all_tests, failed_tests, timedout_tests, hanging_tests] = result
+
     print "Found {} hanging tests:".format(len(hanging_tests))
     for test in hanging_tests:
         print test

http://git-wip-us.apache.org/repos/asf/hbase/blob/f283ff01/dev-support/report-flakies.py
----------------------------------------------------------------------
diff --git a/dev-support/report-flakies.py b/dev-support/report-flakies.py
index 676eca3..20467e7 100755
--- a/dev-support/report-flakies.py
+++ b/dev-support/report-flakies.py
@@ -55,7 +55,11 @@ def get_bad_tests(build_url):
         logger.info("Skipping this build since it is in progress.")
         return {}
     console_url = build_url + "/consoleText"
-    return findHangingTests.get_bad_tests(console_url)
+    result = findHangingTests.get_bad_tests(console_url)
+    if not result:
+        logger.info("Ignoring build {}".format(build_url))
+        return {}
+    return result
 
 
 # If any url is of type multi-configuration project (i.e. has key 'activeConfigurations'),


[20/30] hbase git commit: HBASE-15950 Fix memstore size estimates to be more tighter

Posted by sy...@apache.org.
HBASE-15950 Fix memstore size estimates to be more tighter


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

Branch: refs/heads/hbase-12439
Commit: bff35d6387e810bbd79847efceb3c17b65fdb52d
Parents: 5f9d1a7
Author: Enis Soztutar <en...@apache.org>
Authored: Wed Jun 15 12:13:23 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Wed Jun 15 13:34:39 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/KeyValue.java  |  14 +-
 .../org/apache/hadoop/hbase/util/ClassSize.java | 142 +++++++++++++++----
 .../hbase/regionserver/CompactingMemStore.java  |   1 -
 .../hadoop/hbase/regionserver/HRegion.java      |   4 +-
 .../hbase/TestPartialResultsFromClientSide.java |   9 +-
 .../apache/hadoop/hbase/io/TestHeapSize.java    |  36 +++--
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |   4 +-
 .../regionserver/TestCompactingMemStore.java    |  39 +++--
 .../TestWalAndCompactingMemStoreFlush.java      |  32 +----
 9 files changed, 190 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/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 f741f2c..c1734cc 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
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.RawComparator;
-
 import com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -2598,11 +2597,18 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     int sum = 0;
     sum += ClassSize.OBJECT;// the KeyValue object itself
     sum += ClassSize.REFERENCE;// pointer to "bytes"
-    sum += ClassSize.align(ClassSize.ARRAY);// "bytes"
-    sum += ClassSize.align(length);// number of bytes of data in the "bytes" array
     sum += 2 * Bytes.SIZEOF_INT;// offset, length
     sum += Bytes.SIZEOF_LONG;// memstoreTS
-    return ClassSize.align(sum);
+
+    /*
+     * Deep object overhead for this KV consists of two parts. The first part is the KV object
+     * itself, while the second part is the backing byte[]. We will only count the array overhead
+     * from the byte[] only if this is the first KV in there.
+     */
+    return ClassSize.align(sum) +
+        (offset == 0
+          ? ClassSize.sizeOf(bytes, length) // count both length and object overhead
+          : length);                        // only count the number of bytes
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
index 3dce955..41c93ea 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
@@ -39,8 +39,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 public class ClassSize {
   private static final Log LOG = LogFactory.getLog(ClassSize.class);
 
-  private static int nrOfRefsPerObj = 2;
-
   /** Array overhead */
   public static final int ARRAY;
 
@@ -127,34 +125,124 @@ public class ClassSize {
   }
 
   /**
+   * MemoryLayout abstracts details about the JVM object layout. Default implementation is used in
+   * case Unsafe is not available.
+   */
+  private static class MemoryLayout {
+    int headerSize() {
+      return 2 * oopSize();
+    }
+
+    int arrayHeaderSize() {
+      return (int) align(3 * oopSize());
+    }
+
+    /**
+     * Return the size of an "ordinary object pointer". Either 4 or 8, depending on 32/64 bit,
+     * and CompressedOops
+     */
+    int oopSize() {
+      return is32BitJVM() ? 4 : 8;
+    }
+
+    /**
+     * Aligns a number to 8.
+     * @param num number to align to 8
+     * @return smallest number &gt;= input that is a multiple of 8
+     */
+    public long align(long num) {
+      //The 7 comes from that the alignSize is 8 which is the number of bytes
+      //stored and sent together
+      return  ((num + 7) >> 3) << 3;
+    }
+
+    long sizeOf(byte[] b, int len) {
+      return align(arrayHeaderSize() + len);
+    }
+  }
+
+  /**
+   * UnsafeLayout uses Unsafe to guesstimate the object-layout related parameters like object header
+   * sizes and oop sizes
+   * See HBASE-15950.
+   */
+  private static class UnsafeLayout extends MemoryLayout {
+    @SuppressWarnings("unused")
+    private static final class HeaderSize {
+      private byte a;
+    }
+
+    public UnsafeLayout() {
+    }
+
+    @Override
+    int headerSize() {
+      try {
+        return (int) UnsafeAccess.theUnsafe.objectFieldOffset(
+          HeaderSize.class.getDeclaredField("a"));
+      } catch (NoSuchFieldException | SecurityException e) {
+        LOG.error(e);
+      }
+      return super.headerSize();
+    }
+
+    @Override
+    int arrayHeaderSize() {
+      return UnsafeAccess.theUnsafe.arrayBaseOffset(byte[].class);
+    }
+
+    @Override
+    @SuppressWarnings("static-access")
+    int oopSize() {
+      // Unsafe.addressSize() returns 8, even with CompressedOops. This is how many bytes each
+      // element is allocated in an Object[].
+      return UnsafeAccess.theUnsafe.ARRAY_OBJECT_INDEX_SCALE;
+    }
+
+    @Override
+    @SuppressWarnings("static-access")
+    long sizeOf(byte[] b, int len) {
+      return align(arrayHeaderSize() + len * UnsafeAccess.theUnsafe.ARRAY_BYTE_INDEX_SCALE);
+    }
+  }
+
+  private static MemoryLayout getMemoryLayout() {
+    // Have a safeguard in case Unsafe estimate is wrong. This is static context, there is
+    // no configuration, so we look at System property.
+    String enabled = System.getProperty("hbase.memorylayout.use.unsafe");
+    if (UnsafeAvailChecker.isAvailable() && (enabled == null || Boolean.parseBoolean(enabled))) {
+      LOG.debug("Using Unsafe to estimate memory layout");
+      return new UnsafeLayout();
+    }
+    LOG.debug("Not using Unsafe to estimate memory layout");
+    return new MemoryLayout();
+  }
+
+  private static final MemoryLayout memoryLayout = getMemoryLayout();
+
+  /**
    * Method for reading the arc settings and setting overheads according
    * to 32-bit or 64-bit architecture.
    */
   static {
-    //Default value is set to 8, covering the case when arcModel is unknown
-    if (is32BitJVM()) {
-      REFERENCE = 4;
-    } else {
-      REFERENCE = 8;
-    }
+    REFERENCE = memoryLayout.oopSize();
 
-    OBJECT = 2 * REFERENCE;
+    OBJECT = memoryLayout.headerSize();
 
-    ARRAY = align(3 * REFERENCE);
+    ARRAY = memoryLayout.arrayHeaderSize();
 
-    ARRAYLIST = align(OBJECT + align(REFERENCE) + align(ARRAY) +
-        (2 * Bytes.SIZEOF_INT));
+    ARRAYLIST = align(OBJECT + REFERENCE + (2 * Bytes.SIZEOF_INT)) + align(ARRAY);
 
     //noinspection PointlessArithmeticExpression
-    BYTE_BUFFER = align(OBJECT + align(REFERENCE) + align(ARRAY) +
+    BYTE_BUFFER = align(OBJECT + REFERENCE +
         (5 * Bytes.SIZEOF_INT) +
-        (3 * Bytes.SIZEOF_BOOLEAN) + Bytes.SIZEOF_LONG);
+        (3 * Bytes.SIZEOF_BOOLEAN) + Bytes.SIZEOF_LONG) + align(ARRAY);
 
     INTEGER = align(OBJECT + Bytes.SIZEOF_INT);
 
     MAP_ENTRY = align(OBJECT + 5 * REFERENCE + Bytes.SIZEOF_BOOLEAN);
 
-    TREEMAP = align(OBJECT + (2 * Bytes.SIZEOF_INT) + align(7 * REFERENCE));
+    TREEMAP = align(OBJECT + (2 * Bytes.SIZEOF_INT) + 7 * REFERENCE);
 
     // STRING is different size in jdk6 and jdk7. Just use what we estimate as size rather than
     // have a conditional on whether jdk7.
@@ -174,9 +262,9 @@ public class ClassSize {
     // The size changes from jdk7 to jdk8, estimate the size rather than use a conditional
     CONCURRENT_SKIPLISTMAP = (int) estimateBase(ConcurrentSkipListMap.class, false);
 
-    CONCURRENT_SKIPLISTMAP_ENTRY = align(
+    CONCURRENT_SKIPLISTMAP_ENTRY =
         align(OBJECT + (3 * REFERENCE)) + /* one node per entry */
-        align((OBJECT + (3 * REFERENCE))/2)); /* one index per two entries */
+        align((OBJECT + (3 * REFERENCE))/2); /* one index per two entries */
 
     REENTRANT_LOCK = align(OBJECT + (3 * REFERENCE));
 
@@ -218,8 +306,7 @@ public class ClassSize {
   private static int [] getSizeCoefficients(Class cl, boolean debug) {
     int primitives = 0;
     int arrays = 0;
-    //The number of references that a new object takes
-    int references = nrOfRefsPerObj;
+    int references = 0;
     int index = 0;
 
     for ( ; null != cl; cl = cl.getSuperclass()) {
@@ -275,15 +362,14 @@ public class ClassSize {
    * @return the size estimate, in bytes
    */
   private static long estimateBaseFromCoefficients(int [] coeff, boolean debug) {
-    long prealign_size = coeff[0] + align(coeff[1] * ARRAY) + coeff[2] * REFERENCE;
+    long prealign_size = OBJECT + coeff[0] + coeff[2] * REFERENCE;
 
     // Round up to a multiple of 8
-    long size = align(prealign_size);
-    if(debug) {
+    long size = align(prealign_size) + align(coeff[1] * ARRAY);
+    if (debug) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Primitives=" + coeff[0] + ", arrays=" + coeff[1] +
-            ", references(includes " + nrOfRefsPerObj +
-            " for object overhead)=" + coeff[2] + ", refSize " + REFERENCE +
+            ", references=" + coeff[2] + ", refSize " + REFERENCE +
             ", size=" + size + ", prealign_size=" + prealign_size);
       }
     }
@@ -321,9 +407,7 @@ public class ClassSize {
    * @return smallest number &gt;= input that is a multiple of 8
    */
   public static long align(long num) {
-    //The 7 comes from that the alignSize is 8 which is the number of bytes
-    //stored and sent together
-    return  ((num + 7) >> 3) << 3;
+    return memoryLayout.align(num);
   }
 
   /**
@@ -335,5 +419,9 @@ public class ClassSize {
     return model != null && model.equals("32");
   }
 
+  public static long sizeOf(byte[] b, int len) {
+    return memoryLayout.sizeOf(b, len);
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
index 7aaece6..d47ac36 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
 
 import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.ThreadPoolExecutor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/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 9c966cd..4b4ba9b 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
@@ -5262,8 +5262,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public void releaseRowLocks(List<RowLock> rowLocks) {
     if (rowLocks != null) {
-      for (RowLock rowLock : rowLocks) {
-        rowLock.release();
+      for (int i = 0; i < rowLocks.size(); i++) {
+        rowLocks.get(i).release();
       }
       rowLocks.clear();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index c6a2525..aea92bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.filter.FirstKeyValueMatchingQualifiersFilter;
 import org.apache.hadoop.hbase.filter.RandomRowFilter;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -362,7 +363,7 @@ public class TestPartialResultsFromClientSide {
     if (CELL_HEAP_SIZE == -1) {
       // Do a partial scan that will return a single result with a single cell
       Scan scan = new Scan();
-      scan.setMaxResultSize(1);
+      scan.setMaxResultSize(2);
       scan.setAllowPartialResults(true);
       ResultScanner scanner = TABLE.getScanner(scan);
 
@@ -372,7 +373,11 @@ public class TestPartialResultsFromClientSide {
       assertTrue(result.rawCells() != null);
       assertTrue(result.rawCells().length == 1);
 
-      CELL_HEAP_SIZE = CellUtil.estimatedHeapSizeOf(result.rawCells()[0]);
+      // Estimate the cell heap size. One difference is that on server side, the KV Heap size is
+      // estimated differently in case the cell is backed up by MSLAB byte[] (no overhead for
+      // backing array). Thus below calculation is a bit brittle.
+      CELL_HEAP_SIZE = CellUtil.estimatedHeapSizeOf(result.rawCells()[0])
+          - (ClassSize.ARRAY+3);
       if (LOG.isInfoEnabled()) LOG.info("Cell heap size: " + CELL_HEAP_SIZE);
       scanner.close();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
index 4a4b0e9..09e2271 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.util.ClassSize;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
@@ -57,6 +56,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Testing the sizing that HeapSize offers and compares to the size given by
@@ -68,17 +68,17 @@ public class TestHeapSize  {
   // List of classes implementing HeapSize
   // BatchOperation, BatchUpdate, BlockIndex, Entry, Entry<K,V>, HStoreKey
   // KeyValue, LruBlockCache, LruHashMap<K,V>, Put, WALKey
-  
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     // Print detail on jvm so we know what is different should below test fail.
     RuntimeMXBean b = ManagementFactory.getRuntimeMXBean();
-    LOG.info("name=" + b.getName()); 
-    LOG.info("specname=" + b.getSpecName()); 
-    LOG.info("specvendor=" + b.getSpecVendor()); 
+    LOG.info("name=" + b.getName());
+    LOG.info("specname=" + b.getSpecName());
+    LOG.info("specvendor=" + b.getSpecVendor());
     LOG.info("vmname=" + b.getVmName());
-    LOG.info("vmversion=" + b.getVmVersion()); 
-    LOG.info("vmvendor=" + b.getVmVendor()); 
+    LOG.info("vmversion=" + b.getVmVersion());
+    LOG.info("vmvendor=" + b.getVmVendor());
     Map<String, String> p = b.getSystemProperties();
     LOG.info("properties=" + p);
   }
@@ -132,7 +132,7 @@ public class TestHeapSize  {
     // Object
     cl = Object.class;
     expected = ClassSize.estimateBase(cl, false);
-    actual = ClassSize.OBJECT;
+    actual = ClassSize.align(ClassSize.OBJECT);
     if(expected != actual) {
       ClassSize.estimateBase(cl, true);
       assertEquals(expected, actual);
@@ -390,5 +390,25 @@ public class TestHeapSize  {
     }
   }
 
+  @Test
+  public void testReferenceSize() {
+    LOG.info("ClassSize.REFERENCE is " + ClassSize.REFERENCE);
+    // oop should be either 4 or 8
+    assertTrue(ClassSize.REFERENCE == 4 || ClassSize.REFERENCE == 8);
+  }
+
+  @Test
+  public void testObjectSize() throws IOException {
+    LOG.info("header:" + ClassSize.OBJECT);
+    LOG.info("array header:" + ClassSize.ARRAY);
+
+    if (ClassSize.is32BitJVM()) {
+      assertEquals(ClassSize.OBJECT, 8);
+    } else {
+      assertTrue(ClassSize.OBJECT == 12 || ClassSize.OBJECT == 16); // depending on CompressedOops
+    }
+    assertEquals(ClassSize.OBJECT + 4, ClassSize.ARRAY);
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index eb87a0c..c75232a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -823,9 +823,9 @@ public class TestHFileBlock {
 
   protected void testBlockHeapSizeInternals() {
     if (ClassSize.is32BitJVM()) {
-      assertTrue(HFileBlock.MULTI_BYTE_BUFFER_HEAP_SIZE == 64);
+      assertEquals(64, HFileBlock.MULTI_BYTE_BUFFER_HEAP_SIZE);
     } else {
-      assertTrue(HFileBlock.MULTI_BYTE_BUFFER_HEAP_SIZE == 104);
+      assertEquals(72, HFileBlock.MULTI_BYTE_BUFFER_HEAP_SIZE);
     }
 
     for (int size : new int[] { 100, 256, 12345 }) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index 5c0e42b..c5aae00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -76,6 +76,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     chunkPool.clearChunks();
   }
 
+  @Override
   @Before
   public void setUp() throws Exception {
     super.internalSetUp();
@@ -101,6 +102,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    * @throws IOException
    * @throws CloneNotSupportedException
    */
+  @Override
   @Test
   public void testScanAcrossSnapshot2() throws IOException, CloneNotSupportedException {
     // we are going to the scanning across snapshot with two kvs
@@ -139,6 +141,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    * Test memstore snapshots
    * @throws IOException
    */
+  @Override
   @Test
   public void testSnapshotting() throws IOException {
     final int snapshotCount = 5;
@@ -158,6 +161,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
   /** Test getNextRow from memstore
    * @throws InterruptedException
    */
+  @Override
   @Test
   public void testGetNextRow() throws Exception {
     addRows(this.memstore);
@@ -205,6 +209,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     }
   }
 
+  @Override
   @Test
   public void testGet_memstoreAndSnapShot() throws IOException {
     byte[] row = Bytes.toBytes("testrow");
@@ -251,6 +256,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    * This causes OOME pretty quickly if we use MSLAB for upsert
    * since each 2M chunk is held onto by a single reference.
    */
+  @Override
   @Test
   public void testUpsertMSLAB() throws Exception {
 
@@ -289,6 +295,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    *
    * @throws Exception
    */
+  @Override
   @Test
   public void testUpsertMemstoreSize() throws Exception {
     long oldSize = memstore.size();
@@ -327,6 +334,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    * various edit operations in memstore.
    * @throws Exception
    */
+  @Override
   @Test
   public void testUpdateToTimeOfOldestEdit() throws Exception {
     try {
@@ -581,7 +589,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
 
     // test 1 bucket
     addRowsByKeys(memstore, keys1);
-    assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     long size = memstore.getFlushableSize();
     ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
@@ -589,7 +597,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       Threads.sleep(10);
     }
     assertEquals(0, memstore.getSnapshot().getCellsCount());
-    assertEquals(528, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(376, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     size = memstore.getFlushableSize();
     MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
@@ -608,7 +616,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     String[] keys2 = { "A", "B", "D" };
 
     addRowsByKeys(memstore, keys1);
-    assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize());
+
+    assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     long size = memstore.getFlushableSize();
     ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
@@ -616,10 +625,10 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       Threads.sleep(1000);
     }
     assertEquals(0, memstore.getSnapshot().getCellsCount());
-    assertEquals(528, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(376, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     addRowsByKeys(memstore, keys2);
-    assertEquals(1056, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(752, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     size = memstore.getFlushableSize();
     ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
@@ -627,7 +636,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       Threads.sleep(10);
     }
     assertEquals(0, memstore.getSnapshot().getCellsCount());
-    assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     size = memstore.getFlushableSize();
     MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
@@ -647,7 +656,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     String[] keys3 = { "D", "B", "B" };
 
     addRowsByKeys(memstore, keys1);
-    assertEquals(704, region.getMemstoreSize());
+    assertEquals(496, region.getMemstoreSize());
 
     long size = memstore.getFlushableSize();
     ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
@@ -658,7 +667,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       Threads.sleep(10);
     }
     assertEquals(0, memstore.getSnapshot().getCellsCount());
-    assertEquals(528, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(376, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     addRowsByKeys(memstore, keys2);
 
@@ -666,16 +675,16 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
         region.getMemstoreSize() + ", Memstore Total Size: " +
         regionServicesForStores.getGlobalMemstoreTotalSize() + "\n\n";
 
-    assertEquals(1056, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(752, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     ((CompactingMemStore)memstore).disableCompaction();
     size = memstore.getFlushableSize();
     ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline without compaction
     assertEquals(0, memstore.getSnapshot().getCellsCount());
-    assertEquals(1056, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(752, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     addRowsByKeys(memstore, keys3);
-    assertEquals(1584, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(1128, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     ((CompactingMemStore)memstore).enableCompaction();
     size = memstore.getFlushableSize();
@@ -684,7 +693,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       Threads.sleep(10);
     }
     assertEquals(0, memstore.getSnapshot().getCellsCount());
-    assertEquals(704, regionServicesForStores.getGlobalMemstoreTotalSize());
+    assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize());
 
     size = memstore.getFlushableSize();
     MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
@@ -701,6 +710,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
   private void addRowsByKeys(final AbstractMemStore hmc, String[] keys) {
     byte[] fam = Bytes.toBytes("testfamily");
     byte[] qf = Bytes.toBytes("testqualifier");
+    long size = hmc.getActive().getSize();
     for (int i = 0; i < keys.length; i++) {
       long timestamp = System.currentTimeMillis();
       Threads.sleep(1); // to make sure each kv gets a different ts
@@ -708,10 +718,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       byte[] val = Bytes.toBytes(keys[i] + i);
       KeyValue kv = new KeyValue(row, fam, qf, timestamp, val);
       hmc.add(kv);
-      LOG.debug("added kv: " + kv.getKeyString() + ", timestamp" + kv.getTimestamp());
-      long size = AbstractMemStore.heapSizeChange(kv, true);
-      regionServicesForStores.addAndGetGlobalMemstoreSize(size);
+      LOG.debug("added kv: " + kv.getKeyString() + ", timestamp:" + kv.getTimestamp());
     }
+    regionServicesForStores.addAndGetGlobalMemstoreSize(hmc.getActive().getSize() - size);
   }
 
   private class EnvironmentEdgeForMemstoreTest implements EnvironmentEdge {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff35d63/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
index 4173b2a..2acfd12 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java
@@ -72,8 +72,6 @@ public class TestWalAndCompactingMemStoreFlush {
   public static final byte[] FAMILY2 = FAMILIES[1];
   public static final byte[] FAMILY3 = FAMILIES[2];
 
-
-
   private HRegion initHRegion(String callingMethod, Configuration conf) throws IOException {
     int i=0;
     HTableDescriptor htd = new HTableDescriptor(TABLENAME);
@@ -90,8 +88,6 @@ public class TestWalAndCompactingMemStoreFlush {
     return HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
   }
 
-
-
   // A helper function to create puts.
   private Put createPut(int familyNum, int putNum) {
     byte[] qf  = Bytes.toBytes("q" + familyNum);
@@ -102,7 +98,6 @@ public class TestWalAndCompactingMemStoreFlush {
     return p;
   }
 
-
   // A helper function to create double puts, so something can be compacted later.
   private Put createDoublePut(int familyNum, int putNum) {
     byte[] qf  = Bytes.toBytes("q" + familyNum);
@@ -115,16 +110,12 @@ public class TestWalAndCompactingMemStoreFlush {
     return p;
   }
 
-
   // A helper function to create gets.
   private Get createGet(int familyNum, int putNum) {
     byte[] row = Bytes.toBytes("row" + familyNum + "-" + putNum);
     return new Get(row);
   }
 
-
-
-
   // A helper function to verify edits.
   void verifyEdit(int familyNum, int putNum, Table table) throws IOException {
     Result r = table.get(createGet(familyNum, putNum));
@@ -138,10 +129,6 @@ public class TestWalAndCompactingMemStoreFlush {
       Arrays.equals(r.getFamilyMap(family).get(qf), val));
   }
 
-
-
-
-
   @Test(timeout = 180000)
   public void testSelectiveFlushWhenEnabled() throws IOException {
 
@@ -150,7 +137,7 @@ public class TestWalAndCompactingMemStoreFlush {
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024);
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushNonSloppyStoresFirstPolicy.class
         .getName());
-    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 300 *
+    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 200 *
         1024);
     conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
 
@@ -388,14 +375,6 @@ public class TestWalAndCompactingMemStoreFlush {
     HBaseTestingUtility.closeRegionAndWAL(region);
   }
 
-
-
-
-
-
-
-
-
   @Test(timeout = 180000)
   public void testSelectiveFlushWhenEnabledAndWALinCompaction() throws IOException {
     // Set up the configuration
@@ -403,7 +382,7 @@ public class TestWalAndCompactingMemStoreFlush {
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024);
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushNonSloppyStoresFirstPolicy.class
         .getName());
-    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 300 *
+    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 200 *
         1024);
     conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
 
@@ -535,10 +514,6 @@ public class TestWalAndCompactingMemStoreFlush {
     HBaseTestingUtility.closeRegionAndWAL(region);
   }
 
-
-
-
-
   // Find the (first) region which has the specified name.
   private static Pair<Region, HRegionServer> getRegionWithName(TableName tableName) {
     MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
@@ -552,7 +527,6 @@ public class TestWalAndCompactingMemStoreFlush {
     return null;
   }
 
-
   private WAL getWAL(Region region) {
     return ((HRegion)region).getWAL();
   }
@@ -560,6 +534,4 @@ public class TestWalAndCompactingMemStoreFlush {
   private int getNumRolledLogFiles(Region region) {
     return ((FSHLog)getWAL(region)).getNumRolledLogFiles();
   }
-
-
 }


[16/30] hbase git commit: HBASE-15974 Create a ReplicationQueuesClientHBaseImpl

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
index bd6d070..346ff37 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
@@ -39,6 +39,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -58,15 +59,20 @@ public class TestReplicationStateHBaseImpl {
   private static ReplicationQueues rq1;
   private static ReplicationQueues rq2;
   private static ReplicationQueues rq3;
+  private static ReplicationQueuesClient rqc;
   private static ReplicationPeers rp;
 
-  private static final String server1 = ServerName.valueOf("hostname1.example.org", 1234, 123L)
-      .toString();
+
+  private static final String server0 = ServerName.valueOf("hostname0.example.org", 1234, -1L)
+    .toString();
+  private static final String server1 = ServerName.valueOf("hostname1.example.org", 1234, 1L)
+    .toString();
   private static final String server2 = ServerName.valueOf("hostname2.example.org", 1234, 1L)
-      .toString();
+    .toString();
   private static final String server3 = ServerName.valueOf("hostname3.example.org", 1234, 1L)
-      .toString();
+    .toString();
 
+  private static DummyServer ds0;
   private static DummyServer ds1;
   private static DummyServer ds2;
   private static DummyServer ds3;
@@ -77,9 +83,9 @@ public class TestReplicationStateHBaseImpl {
     utility.startMiniCluster();
     conf = utility.getConfiguration();
     conf.setClass("hbase.region.replica.replication.ReplicationQueuesType",
-        ReplicationQueuesHBaseImpl.class, ReplicationQueues.class);
-    conf.setClass("hbase.region.replica.replication.ReplicationQueuesType",
-      ReplicationQueuesHBaseImpl.class, ReplicationQueues.class);
+      TableBasedReplicationQueuesImpl.class, ReplicationQueues.class);
+    conf.setClass("hbase.region.replica.replication.ReplicationQueuesClientType",
+      TableBasedReplicationQueuesClientImpl.class, ReplicationQueuesClient.class);
     zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
     String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
     replicationZNode = ZKUtil.joinZNode(zkw.baseZNode, replicationZNodeName);
@@ -88,6 +94,9 @@ public class TestReplicationStateHBaseImpl {
   @Before
   public void setUp() {
     try {
+      ds0 = new DummyServer(server0);
+      rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(
+        conf, ds0));
       ds1 = new DummyServer(server1);
       rq1 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds1, zkw));
       rq1.init(server1);
@@ -99,9 +108,6 @@ public class TestReplicationStateHBaseImpl {
       rq3.init(server3);
       rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
       rp.init();
-      rp.addPeer("Queue1", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus1"));
-      rp.addPeer("Queue2", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus2"));
-      rp.addPeer("Queue3", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus3"));
     } catch (Exception e) {
       fail("testReplicationStateHBaseConstruction received an exception" + e.getMessage());
     }
@@ -165,13 +171,13 @@ public class TestReplicationStateHBaseImpl {
       try {
         rq1.getLogPosition("Queue1", "NotHereWAL");
         fail("Replication queue should have thrown a ReplicationException for reading from a " +
-            "non-existent WAL");
+          "non-existent WAL");
       } catch (ReplicationException e) {
       }
       try {
         rq1.getLogPosition("NotHereQueue", "NotHereWAL");
         fail("Replication queue should have thrown a ReplicationException for reading from a " +
-            "non-existent queue");
+          "non-existent queue");
       } catch (ReplicationException e) {
       }
       // Test removing logs
@@ -198,6 +204,13 @@ public class TestReplicationStateHBaseImpl {
   @Test
   public void TestMultipleReplicationQueuesHBaseImpl () {
     try {
+      rp.addPeer("Queue1", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus1"));
+      rp.addPeer("Queue2", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus2"));
+      rp.addPeer("Queue3", new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus3"));
+    } catch (ReplicationException e) {
+      fail("Failed to add peers to ReplicationPeers");
+    }
+    try {
       // Test adding in WAL files
       rq1.addLog("Queue1", "WALLogFile1.1");
       rq1.addLog("Queue1", "WALLogFile1.2");
@@ -298,6 +311,56 @@ public class TestReplicationStateHBaseImpl {
     }
   }
 
+  @Test
+  public void TestReplicationQueuesClient() throws Exception{
+
+    // Test ReplicationQueuesClient log tracking
+    rq1.addLog("Queue1", "WALLogFile1.1");
+    assertEquals(1, rqc.getLogsInQueue(server1, "Queue1").size());
+    rq1.removeLog("Queue1", "WALLogFile1.1");
+    assertEquals(0, rqc.getLogsInQueue(server1, "Queue1").size());
+    rq2.addLog("Queue2", "WALLogFile2.1");
+    rq2.addLog("Queue2", "WALLogFile2.2");
+    assertEquals(2, rqc.getLogsInQueue(server2, "Queue2").size());
+    rq3.addLog("Queue1", "WALLogFile1.1");
+    rq3.addLog("Queue3", "WALLogFile3.1");
+    rq3.addLog("Queue3", "WALLogFile3.2");
+
+    // Test ReplicationQueueClient log tracking for faulty cases
+    assertEquals(0, ds0.getAbortCount());
+    assertNull(rqc.getLogsInQueue("NotHereServer", "NotHereQueue"));
+    assertNull(rqc.getLogsInQueue(server1, "NotHereQueue"));
+    assertNull(rqc.getLogsInQueue("NotHereServer", "WALLogFile1.1"));
+    assertEquals(3, ds0.getAbortCount());
+    // Test ReplicationQueueClient replicators
+    List<String> replicators = rqc.getListOfReplicators();
+    assertEquals(3, replicators.size());
+    assertTrue(replicators.contains(server1));
+    assertTrue(replicators.contains(server2));
+    rq1.removeQueue("Queue1");
+    assertEquals(2, rqc.getListOfReplicators().size());
+
+    // Test ReplicationQueuesClient queue tracking
+    assertEquals(0, rqc.getAllQueues(server1).size());
+    rq1.addLog("Queue2", "WALLogFile2.1");
+    rq1.addLog("Queue3", "WALLogFile3.1");
+    assertEquals(2, rqc.getAllQueues(server1).size());
+    rq1.removeAllQueues();
+    assertEquals(0, rqc.getAllQueues(server1).size());
+
+    // Test ReplicationQueuesClient queue tracking for faulty cases
+    assertEquals(0, rqc.getAllQueues("NotHereServer").size());
+
+    // Test ReplicationQueuesClient get all WAL's
+    assertEquals(5 , rqc.getAllWALs().size());
+    rq3.removeLog("Queue1", "WALLogFile1.1");
+    assertEquals(4, rqc.getAllWALs().size());
+    rq3.removeAllQueues();
+    assertEquals(2, rqc.getAllWALs().size());
+    rq2.removeAllQueues();
+    assertEquals(0, rqc.getAllWALs().size());
+  }
+
   @After
   public void clearQueues() throws Exception{
     rq1.removeAllQueues();
@@ -306,6 +369,7 @@ public class TestReplicationStateHBaseImpl {
     assertEquals(0, rq1.getAllQueues().size());
     assertEquals(0, rq2.getAllQueues().size());
     assertEquals(0, rq3.getAllQueues().size());
+    ds0.resetAbortCount();
     ds1.resetAbortCount();
     ds2.resetAbortCount();
     ds3.resetAbortCount();
@@ -313,7 +377,7 @@ public class TestReplicationStateHBaseImpl {
 
   @After
   public void tearDown() throws KeeperException, IOException {
-     ZKUtil.deleteNodeRecursively(zkw, replicationZNode);
+    ZKUtil.deleteNodeRecursively(zkw, replicationZNode);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index 972a400..a357a1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -96,11 +96,13 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
       rq1 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds1, zkw));
       rq2 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds2, zkw));
       rq3 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds3, zkw));
+      rqc = ReplicationFactory.getReplicationQueuesClient(
+        new ReplicationQueuesClientArguments(conf, ds1, zkw));
     } catch (Exception e) {
-      // This should not occur, because getReplicationQueues() only throws for ReplicationQueuesHBaseImpl
+      // This should not occur, because getReplicationQueues() only throws for
+      // TableBasedReplicationQueuesImpl
       fail("ReplicationFactory.getReplicationQueues() threw an IO Exception");
     }
-    rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, ds1);
     rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
     OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
     rqZK = new ReplicationQueuesZKImpl(zkw, conf, ds1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2093aade/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 e14fd3c..bf47d4f 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
@@ -68,6 +68,8 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
@@ -436,8 +438,9 @@ public class TestReplicationSourceManager {
           s1.getZooKeeper()));
     rq1.init(s1.getServerName().toString());
 
-    ReplicationQueuesClient client =
-        ReplicationFactory.getReplicationQueuesClient(s1.getZooKeeper(), s1.getConfiguration(), s1);
+    ReplicationQueuesClientZKImpl client =
+        (ReplicationQueuesClientZKImpl)ReplicationFactory.getReplicationQueuesClient(
+        new ReplicationQueuesClientArguments(s1.getConfiguration(), s1, s1.getZooKeeper()));
 
     int v0 = client.getQueuesZNodeCversion();
     rq1.claimQueues(s0.getServerName().getServerName());


[12/30] hbase git commit: HBASE-16023 Fastpath for the FIFO rpcscheduler Adds an executor that does balanced queue and fast path handing off requests directly to waiting handlers if any present. Idea taken from Apace Kudu (incubating). See https://gerrit

Posted by sy...@apache.org.
HBASE-16023 Fastpath for the FIFO rpcscheduler Adds an executor that does balanced queue and fast path handing off requests directly to waiting handlers if any present. Idea taken from Apace Kudu (incubating). See https://gerrit.cloudera.org/#/c/2938/7/src/kudu/rpc/service_queue.h

M hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
 Refactor which makes a Handler type. Put all 'handler' stuff inside this
 new type. Also make it so subclass can provide its own Handler type.

M hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
 Name the handler threads for their type so can tell if configs are
 having an effect.

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


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

Branch: refs/heads/hbase-12439
Commit: ee86e91e7eb3d647c89db496baa12a332f2c0c18
Parents: fa50d45
Author: stack <st...@apache.org>
Authored: Tue Jun 14 11:18:34 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Jun 14 17:42:01 2016 -0700

----------------------------------------------------------------------
 .../hbase/ipc/BalancedQueueRpcExecutor.java     |   5 +-
 ...ifoWithFastPathBalancedQueueRpcExecutor.java | 116 ++++++++++++++
 .../apache/hadoop/hbase/ipc/RpcExecutor.java    | 160 ++++++++++++-------
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    |  39 +++--
 .../hbase/ipc/TestSimpleRpcScheduler.java       |   3 +-
 5 files changed, 241 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ee86e91e/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
index 3505221..241d36e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
@@ -33,7 +33,8 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
 
 /**
  * An {@link RpcExecutor} that will balance requests evenly across all its queues, but still remains
- * efficient with a single queue via an inlinable queue balancing mechanism.
+ * efficient with a single queue via an inlinable queue balancing mechanism. Defaults to FIFO but
+ * you can pass an alternate queue class to use.
  */
 @InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX })
 @InterfaceStability.Evolving
@@ -103,4 +104,4 @@ public class BalancedQueueRpcExecutor extends RpcExecutor {
   public List<BlockingQueue<CallRunner>> getQueues() {
     return queues;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ee86e91e/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoWithFastPathBalancedQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoWithFastPathBalancedQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoWithFastPathBalancedQueueRpcExecutor.java
new file mode 100644
index 0000000..1951dd0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoWithFastPathBalancedQueueRpcExecutor.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.ipc;
+
+import java.util.Deque;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * FIFO balanced queue executor with a fastpath. Because this is FIFO, it has no respect for
+ * ordering so a fast path skipping the queuing of Calls if an Handler is available, is possible.
+ * Just pass the Call direct to waiting Handler thread. Try to keep the hot Handlers bubbling
+ * rather than let them go cold and lose context. Idea taken from Apace Kudu (incubating). See
+ * https://gerrit.cloudera.org/#/c/2938/7/src/kudu/rpc/service_queue.h
+ */
+@InterfaceAudience.Private
+public class FifoWithFastPathBalancedQueueRpcExecutor extends BalancedQueueRpcExecutor {
+  // Depends on default behavior of BalancedQueueRpcExecutor being FIFO!
+
+  /*
+   * Stack of Handlers waiting for work.
+   */
+  private final Deque<FastPathHandler> fastPathHandlerStack = new ConcurrentLinkedDeque<>();
+
+  public FifoWithFastPathBalancedQueueRpcExecutor(final String name, final int handlerCount,
+      final int numQueues, final int maxQueueLength, final Configuration conf,
+      final Abortable abortable) {
+    super(name, handlerCount, numQueues, conf, abortable, LinkedBlockingQueue.class,
+        maxQueueLength);
+  }
+
+  @Override
+  protected Handler getHandler(String name, double handlerFailureThreshhold,
+      BlockingQueue<CallRunner> q) {
+    return new FastPathHandler(name, handlerFailureThreshhold, q, fastPathHandlerStack);
+  }
+
+  @Override
+  public boolean dispatch(CallRunner callTask) throws InterruptedException {
+    FastPathHandler handler = popReadyHandler();
+    return handler != null? handler.loadCallRunner(callTask): super.dispatch(callTask);
+  }
+
+  /**
+   * @return Pop a Handler instance if one available ready-to-go or else return null.
+   */
+  private FastPathHandler popReadyHandler() {
+    return this.fastPathHandlerStack.poll();
+  }
+
+  class FastPathHandler extends Handler {
+    // Below are for fast-path support. Push this Handler on to the fastPathHandlerStack Deque
+    // if an empty queue of CallRunners so we are available for direct handoff when one comes in.
+    final Deque<FastPathHandler> fastPathHandlerStack;
+    // Semaphore to coordinate loading of fastpathed loadedTask and our running it.
+    private Semaphore semaphore = new Semaphore(1);
+    // The task we get when fast-pathing.
+    private CallRunner loadedCallRunner;
+
+    FastPathHandler(String name, double handlerFailureThreshhold, BlockingQueue<CallRunner> q,
+        final Deque<FastPathHandler> fastPathHandlerStack) {
+      super(name, handlerFailureThreshhold, q);
+      this.fastPathHandlerStack = fastPathHandlerStack;
+      this.semaphore.drainPermits();
+    }
+
+    protected CallRunner getCallRunner() throws InterruptedException {
+      // Get a callrunner if one in the Q.
+      CallRunner cr = this.q.poll();
+      if (cr == null) {
+        // Else, if a fastPathHandlerStack present and no callrunner in Q, register ourselves for
+        // the fastpath handoff done via fastPathHandlerStack.
+        if (this.fastPathHandlerStack != null) {
+          this.fastPathHandlerStack.push(this);
+          this.semaphore.acquire();
+          cr = this.loadedCallRunner;
+        } else {
+          // No fastpath available. Block until a task comes available.
+          cr = super.getCallRunner();
+        }
+      }
+      return cr;
+    }
+
+    /**
+     * @param task Task gotten via fastpath.
+     * @return True if we successfully loaded our task
+     */
+    boolean loadCallRunner(final CallRunner cr) {
+      this.loadedCallRunner = cr;
+      this.semaphore.release();
+      return true;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ee86e91e/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 880df36..5b6c6c8 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.ipc;
 
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
@@ -31,15 +32,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 
+/**
+ * Runs the CallRunners passed here via {@link #dispatch(CallRunner)}. Subclass and add particular
+ * scheduling behavior.
+ */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public abstract class RpcExecutor {
   private static final Log LOG = LogFactory.getLog(RpcExecutor.class);
 
@@ -48,7 +51,7 @@ public abstract class RpcExecutor {
   protected volatile int currentQueueLimit;
 
   private final AtomicInteger activeHandlerCount = new AtomicInteger(0);
-  private final List<Thread> handlers;
+  private final List<Handler> handlers;
   private final int handlerCount;
   private final String name;
   private final AtomicInteger failedHandlerCount = new AtomicInteger(0);
@@ -59,7 +62,7 @@ public abstract class RpcExecutor {
   private Abortable abortable = null;
 
   public RpcExecutor(final String name, final int handlerCount) {
-    this.handlers = new ArrayList<Thread>(handlerCount);
+    this.handlers = new ArrayList<Handler>(handlerCount);
     this.handlerCount = handlerCount;
     this.name = Strings.nullToEmpty(name);
   }
@@ -101,75 +104,111 @@ public abstract class RpcExecutor {
     startHandlers(null, handlerCount, callQueues, 0, callQueues.size(), port);
   }
 
+  /**
+   * Override if providing alternate Handler implementation.
+   */
+  protected Handler getHandler(final String name, final double handlerFailureThreshhold,
+      final BlockingQueue<CallRunner> q) {
+    return new Handler(name, handlerFailureThreshhold, q);
+  }
+
+  /**
+   * Start up our handlers.
+   */
   protected void startHandlers(final String nameSuffix, final int numHandlers,
       final List<BlockingQueue<CallRunner>> callQueues,
       final int qindex, final int qsize, final int port) {
     final String threadPrefix = name + Strings.nullToEmpty(nameSuffix);
+    double handlerFailureThreshhold =
+        conf == null ? 1.0 : conf.getDouble(HConstants.REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT,
+          HConstants.DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT);
     for (int i = 0; i < numHandlers; i++) {
       final int index = qindex + (i % qsize);
-      Thread t = new Thread(new Runnable() {
-        @Override
-        public void run() {
-          consumerLoop(callQueues.get(index));
-        }
-      });
-      t.setDaemon(true);
-      t.setName(threadPrefix + "RpcServer.handler=" + handlers.size() +
-        ",queue=" + index + ",port=" + port);
-      t.start();
-      LOG.debug(threadPrefix + " Start Handler index=" + handlers.size() + " queue=" + index);
-      handlers.add(t);
+      String name = "RpcServer." + threadPrefix + ".handler=" + handlers.size() + ",queue=" +
+          index + ",port=" + port;
+      Handler handler = getHandler(name, handlerFailureThreshhold, callQueues.get(index));
+      handler.start();
+      LOG.debug("Started " + name);
+      handlers.add(handler);
     }
   }
 
-  protected void consumerLoop(final BlockingQueue<CallRunner> myQueue) {
-    boolean interrupted = false;
-    double handlerFailureThreshhold =
-        conf == null ? 1.0 : conf.getDouble(HConstants.REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT,
-          HConstants.DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT);
-    try {
-      while (running) {
-        try {
-          MonitoredRPCHandler status = RpcServer.getStatus();
-          CallRunner task = myQueue.take();
-          task.setStatus(status);
+  /**
+   * Handler thread run the {@link CallRunner#run()} in.
+   */
+  protected class Handler extends Thread {
+    /**
+     * Q to find CallRunners to run in.
+     */
+    final BlockingQueue<CallRunner> q;
+
+    final double handlerFailureThreshhold;
+
+    Handler(final String name, final double handlerFailureThreshhold,
+        final BlockingQueue<CallRunner> q) {
+      super(name);
+      setDaemon(true);
+      this.q = q;
+      this.handlerFailureThreshhold = handlerFailureThreshhold;
+    }
+
+    /**
+     * @return A {@link CallRunner}
+     * @throws InterruptedException
+     */
+    protected CallRunner getCallRunner() throws InterruptedException {
+      return this.q.take();
+    }
+
+    @Override
+    public void run() {
+      boolean interrupted = false;
+      try {
+        while (running) {
           try {
-            activeHandlerCount.incrementAndGet();
-            task.run();
-          } catch (Throwable e) {
-            if (e instanceof Error) {
-              int failedCount = failedHandlerCount.incrementAndGet();
-              if (handlerFailureThreshhold >= 0
-                  && failedCount > handlerCount * handlerFailureThreshhold) {
-                String message =
-                    "Number of failed RpcServer handler exceeded threshhold "
-                        + handlerFailureThreshhold + "  with failed reason: "
-                        + StringUtils.stringifyException(e);
-                if (abortable != null) {
-                  abortable.abort(message, e);
-                } else {
-                  LOG.error("Received " + StringUtils.stringifyException(e)
-                    + " but not aborting due to abortable being null");
-                  throw e;
-                }
-              } else {
-                LOG.warn("RpcServer handler threads encountered errors "
-                    + StringUtils.stringifyException(e));
-              }
+            run(getCallRunner());
+          } catch (InterruptedException e) {
+            interrupted = true;
+          }
+        }
+      } catch (Exception e) {
+        LOG.warn(e);
+        throw e;
+      } finally {
+        if (interrupted) {
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+
+    private void run(CallRunner cr) {
+      MonitoredRPCHandler status = RpcServer.getStatus();
+      cr.setStatus(status);
+      try {
+        activeHandlerCount.incrementAndGet();
+        cr.run();
+      } catch (Throwable e) {
+        if (e instanceof Error) {
+          int failedCount = failedHandlerCount.incrementAndGet();
+          if (this.handlerFailureThreshhold >= 0 &&
+              failedCount > handlerCount * this.handlerFailureThreshhold) {
+            String message = "Number of failed RpcServer handler runs exceeded threshhold " +
+              this.handlerFailureThreshhold + "; reason: " + StringUtils.stringifyException(e);
+            if (abortable != null) {
+              abortable.abort(message, e);
             } else {
-              LOG.warn("RpcServer handler threads encountered exceptions "
-                  + StringUtils.stringifyException(e));
+              LOG.error("Error but can't abort because abortable is null: " +
+                  StringUtils.stringifyException(e));
+              throw e;
             }
-          } finally {
-            activeHandlerCount.decrementAndGet();
+          } else {
+            LOG.warn("Handler errors " + StringUtils.stringifyException(e));
           }
-        } catch (InterruptedException e) {
-          interrupted = true;
+        } else {
+          LOG.warn("Handler  exception " + StringUtils.stringifyException(e));
         }
-      }
-    } finally {
-      if (interrupted) {
-        Thread.currentThread().interrupt();
+      } finally {
+        activeHandlerCount.decrementAndGet();
       }
     }
   }
@@ -194,7 +233,6 @@ public abstract class RpcExecutor {
    * All requests go to the first queue, at index 0
    */
   private static QueueBalancer ONE_QUEUE = new QueueBalancer() {
-
     @Override
     public int getNextQueue() {
       return 0;
@@ -227,4 +265,4 @@ public abstract class RpcExecutor {
     }
     currentQueueLimit = conf.getInt(configKey, currentQueueLimit);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ee86e91e/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index 85bf78d..6e623a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -99,7 +99,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY,
       CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
 
-    if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
+    if (isCodelQueueType(callQueueType)) {
       // update CoDel Scheduler tunables
       int codelTargetDelay = conf.getInt(CALL_QUEUE_CODEL_TARGET_DELAY,
         CALL_QUEUE_CODEL_DEFAULT_TARGET_DELAY);
@@ -204,18 +204,19 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
       // multiple read/write queues
       if (isDeadlineQueueType(callQueueType)) {
         CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
-        callExecutor = new RWQueueRpcExecutor("RW.deadline.Q", handlerCount, numCallQueues,
+        callExecutor = new RWQueueRpcExecutor("DeadlineRWQ.default", handlerCount, numCallQueues,
             callqReadShare, callqScanShare, maxQueueLength, conf, abortable,
             BoundedPriorityBlockingQueue.class, callPriority);
-      } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
+      } else if (isCodelQueueType(callQueueType)) {
         Object[] callQueueInitArgs = {maxQueueLength, codelTargetDelay, codelInterval,
           codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches};
-        callExecutor = new RWQueueRpcExecutor("RW.codel.Q", handlerCount,
+        callExecutor = new RWQueueRpcExecutor("CodelRWQ.default", handlerCount,
           numCallQueues, callqReadShare, callqScanShare,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs);
       } else {
-        callExecutor = new RWQueueRpcExecutor("RW.fifo.Q", handlerCount, numCallQueues,
+        // FifoWFPBQ = FifoWithFastPathBalancedQueueRpcExecutor
+        callExecutor = new RWQueueRpcExecutor("FifoRWQ.default", handlerCount, numCallQueues,
           callqReadShare, callqScanShare, maxQueueLength, conf, abortable);
       }
     } else {
@@ -223,33 +224,37 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
       if (isDeadlineQueueType(callQueueType)) {
         CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
         callExecutor =
-          new BalancedQueueRpcExecutor("B.deadline.Q", handlerCount, numCallQueues,
+          new BalancedQueueRpcExecutor("DeadlineBQ.default", handlerCount, numCallQueues,
             conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
-      } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
+      } else if (isCodelQueueType(callQueueType)) {
         callExecutor =
-          new BalancedQueueRpcExecutor("B.codel.Q", handlerCount, numCallQueues,
+          new BalancedQueueRpcExecutor("CodelBQ.default", handlerCount, numCallQueues,
             conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
             codelTargetDelay, codelInterval, codelLifoThreshold,
             numGeneralCallsDropped, numLifoModeSwitches);
       } else {
-        callExecutor = new BalancedQueueRpcExecutor("B.fifo.Q", handlerCount,
-            numCallQueues, maxQueueLength, conf, abortable);
+        // FifoWFPBQ = FifoWithFastPathBalancedQueueRpcExecutor
+        callExecutor = new FifoWithFastPathBalancedQueueRpcExecutor("FifoWFPBQ.default",
+            handlerCount, numCallQueues, maxQueueLength, conf, abortable);
       }
     }
     // Create 2 queues to help priorityExecutor be more scalable.
-    this.priorityExecutor = priorityHandlerCount > 0 ?
-      new BalancedQueueRpcExecutor("B.priority.fifo.Q", priorityHandlerCount, 2,
-          maxPriorityQueueLength):
-      null;
-   this.replicationExecutor =
-     replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("B.replication.fifo.Q",
-       replicationHandlerCount, 1, maxQueueLength, conf, abortable) : null;
+    this.priorityExecutor = priorityHandlerCount > 0?
+      new FifoWithFastPathBalancedQueueRpcExecutor("FifoWFPBQ.priority", priorityHandlerCount,
+         2, maxPriorityQueueLength, conf, abortable): null;
+    this.replicationExecutor = replicationHandlerCount > 0?
+      new FifoWithFastPathBalancedQueueRpcExecutor("FifoWFPBQ.replication",
+        replicationHandlerCount, 1, maxQueueLength, conf, abortable) : null;
   }
 
   private static boolean isDeadlineQueueType(final String callQueueType) {
     return callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
   }
 
+  private static boolean isCodelQueueType(final String callQueueType) {
+    return callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE);
+  }
+
   public SimpleRpcScheduler(
 	      Configuration conf,
 	      int handlerCount,

http://git-wip-us.apache.org/repos/asf/hbase/blob/ee86e91e/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 8ae3078..53f9175 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
@@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.testclassification.RPCTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
@@ -242,7 +241,7 @@ public class TestSimpleRpcScheduler {
       // -> WITH REORDER [10 10 10 10 10 10 50 100] -> 530 (Deadline Queue)
       if (queueType.equals(SimpleRpcScheduler.CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
         assertEquals(530, totalTime);
-      } else /* if (queueType.equals(SimpleRpcScheduler.CALL_QUEUE_TYPE_FIFO_CONF_VALUE)) */ {
+      } else if (queueType.equals(SimpleRpcScheduler.CALL_QUEUE_TYPE_FIFO_CONF_VALUE)) {
         assertEquals(930, totalTime);
       }
     } finally {


[02/30] hbase git commit: HBASE-16007 Job's Configuration should be passed to TableMapReduceUtil#addDependencyJars() in WALPlayer

Posted by sy...@apache.org.
HBASE-16007 Job's Configuration should be passed to TableMapReduceUtil#addDependencyJars() in WALPlayer


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

Branch: refs/heads/hbase-12439
Commit: 6360cc0307518501faac114a9cce58ef19eeca0c
Parents: 4c885f4
Author: tedyu <yu...@gmail.com>
Authored: Sat Jun 11 07:05:27 2016 -0700
Committer: Ted <yu...@gmail.com>
Committed: Sat Jun 11 07:05:27 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/6360cc03/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index d36721e..c6fefb2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -295,7 +295,7 @@ public class WALPlayer extends Configured implements Tool {
     }
     String codecCls = WALCellCodec.getWALCellCodecClass(conf);
     try {
-      TableMapReduceUtil.addDependencyJars(conf, Class.forName(codecCls));
+      TableMapReduceUtil.addDependencyJars(job.getConfiguration(), Class.forName(codecCls));
     } catch (Exception e) {
       throw new IOException("Cannot determine wal codec class " + codecCls, e);
     }


[29/30] hbase git commit: HBASE-5291 Addendum 2 passes correct path to deleteRecursively

Posted by sy...@apache.org.
HBASE-5291 Addendum 2 passes correct path to deleteRecursively


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

Branch: refs/heads/hbase-12439
Commit: 6d0e0e3721fd7a0c020ce5c746c9369cb4220393
Parents: d8902ba
Author: tedyu <yu...@gmail.com>
Authored: Thu Jun 16 17:00:57 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Jun 16 17:00:57 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/6d0e0e37/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
index 7e773b7..a64c1e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
@@ -247,7 +247,7 @@ public class HttpServerFunctionalTest extends Assert {
         if (child.isFile()) {
           child.delete();
         } else {
-          deleteRecursively(d);
+          deleteRecursively(child);
         }
       }
     }


[13/30] hbase git commit: HBASE-16023 Fastpath for the FIFO rpcscheduler. addendum

Posted by sy...@apache.org.
HBASE-16023 Fastpath for the FIFO rpcscheduler. addendum


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

Branch: refs/heads/hbase-12439
Commit: db234bf15dac9c7904a2ad50a86ac4f1a7da3ca1
Parents: ee86e91
Author: chenheng <ch...@apache.org>
Authored: Wed Jun 15 16:15:41 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Wed Jun 15 16:20:55 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/db234bf1/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 53f9175..0e50761 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.RPCTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;


[27/30] hbase git commit: HBASE-16047 TestFastFail is broken again

Posted by sy...@apache.org.
HBASE-16047 TestFastFail is broken again


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

Branch: refs/heads/hbase-12439
Commit: 62a4a2c02985fd7a091c441f10eacc10e29d0395
Parents: 6d02f36
Author: Mikhail Antonov <an...@apache.org>
Authored: Thu Jun 16 14:05:56 2016 -0700
Committer: Mikhail Antonov <an...@apache.org>
Committed: Thu Jun 16 14:09:48 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/62a4a2c0/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
index 07f1948..89b28fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
@@ -71,6 +71,9 @@ public class TestFastFail {
    */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    // Just to prevent fastpath FIFO from picking calls up bypassing the queue.
+    TEST_UTIL.getConfiguration().set(
+      SimpleRpcScheduler.CALL_QUEUE_TYPE_CONF_KEY, "deadline");
     TEST_UTIL.startMiniCluster(SLAVES);
   }
 


[04/30] hbase git commit: Revert HBASE-15991 commit

Posted by sy...@apache.org.
Revert HBASE-15991 commit


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

Branch: refs/heads/hbase-12439
Commit: 2d0448fa84745991d2447ff78600866873b1fec0
Parents: 2738518
Author: Ramkrishna <ra...@intel.com>
Authored: Mon Jun 13 15:05:50 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Mon Jun 13 15:05:50 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/RegionServicesForStores.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0448fa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
index e481a63..72f7bf5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -43,7 +42,7 @@ public class RegionServicesForStores {
   private static final int POOL_SIZE = 10;
   private static final ThreadPoolExecutor INMEMORY_COMPACTION_POOL =
       new ThreadPoolExecutor(POOL_SIZE, POOL_SIZE, 60, TimeUnit.SECONDS,
-          new LinkedBlockingQueue<Runnable>(),
+          new StealJobQueue<Runnable>().getStealFromQueue(),
           new ThreadFactory() {
             @Override
             public Thread newThread(Runnable r) {


[23/30] hbase git commit: HBASE-15977 Failed variable substitution on home page

Posted by sy...@apache.org.
HBASE-15977 Failed variable substitution on home page

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

Branch: refs/heads/hbase-12439
Commit: 158568e7806e461275406bc15856ba26e4660f4c
Parents: f283ff0
Author: Dima Spivak <ds...@cloudera.com>
Authored: Tue Jun 14 22:36:57 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Jun 15 16:03:45 2016 -0700

----------------------------------------------------------------------
 src/main/site/site.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/158568e7/src/main/site/site.xml
----------------------------------------------------------------------
diff --git a/src/main/site/site.xml b/src/main/site/site.xml
index fb237bb..38dc0c1 100644
--- a/src/main/site/site.xml
+++ b/src/main/site/site.xml
@@ -43,6 +43,7 @@
     </fluidoSkin>
   </custom>
   <bannerLeft>
+    <name/>
     <height>0</height>
     <width>0</width>
   </bannerLeft>