You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2016/01/14 18:08:15 UTC

[01/50] [abbrv] hbase git commit: HBASE-15023 Reenable TestShell and TestStochasticLoadBalancer

Repository: hbase
Updated Branches:
  refs/heads/trunk bbfff0d07 -> dc57996ca


HBASE-15023 Reenable TestShell and TestStochasticLoadBalancer


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

Branch: refs/heads/trunk
Commit: 2fba25b66aabcee199719114eebafdcaf6cc2942
Parents: a999c2a
Author: stack <st...@apache.org>
Authored: Mon Dec 28 13:27:17 2015 -0800
Committer: stack <st...@apache.org>
Committed: Mon Dec 28 13:27:17 2015 -0800

----------------------------------------------------------------------
 .../balancer/TestStochasticLoadBalancer.java    | 532 +++++++++++++++++++
 .../apache/hadoop/hbase/client/TestShell.java   |  39 ++
 2 files changed, 571 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2fba25b6/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
new file mode 100644
index 0000000..7abbeb4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
@@ -0,0 +1,532 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.balancer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.master.RackManager;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
+import org.apache.hadoop.hbase.testclassification.FlakeyTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({FlakeyTests.class, MediumTests.class})
+public class TestStochasticLoadBalancer extends BalancerTestBase {
+  public static final String REGION_KEY = "testRegion";
+  private static final Log LOG = LogFactory.getLog(TestStochasticLoadBalancer.class);
+
+  @Test
+  public void testKeepRegionLoad() throws Exception {
+
+    ServerName sn = ServerName.valueOf("test:8080", 100);
+    int numClusterStatusToAdd = 20000;
+    for (int i = 0; i < numClusterStatusToAdd; i++) {
+      ServerLoad sl = mock(ServerLoad.class);
+
+      RegionLoad rl = mock(RegionLoad.class);
+      when(rl.getStores()).thenReturn(i);
+
+      Map<byte[], RegionLoad> regionLoadMap =
+          new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR);
+      regionLoadMap.put(Bytes.toBytes(REGION_KEY), rl);
+      when(sl.getRegionsLoad()).thenReturn(regionLoadMap);
+
+      ClusterStatus clusterStatus = mock(ClusterStatus.class);
+      when(clusterStatus.getServers()).thenReturn(Arrays.asList(sn));
+      when(clusterStatus.getLoad(sn)).thenReturn(sl);
+
+      loadBalancer.setClusterStatus(clusterStatus);
+    }
+    assertTrue(loadBalancer.loads.get(REGION_KEY) != null);
+    assertTrue(loadBalancer.loads.get(REGION_KEY).size() == 15);
+
+    Queue<RegionLoad> loads = loadBalancer.loads.get(REGION_KEY);
+    int i = 0;
+    while(loads.size() > 0) {
+      RegionLoad rl = loads.remove();
+      assertEquals(i + (numClusterStatusToAdd - 15), rl.getStores());
+      i ++;
+    }
+  }
+
+  /**
+   * Test the load balancing algorithm.
+   *
+   * Invariant is that all servers should be hosting either floor(average) or
+   * ceiling(average)
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testBalanceCluster() throws Exception {
+
+    for (int[] mockCluster : clusterStateMocks) {
+      Map<ServerName, List<HRegionInfo>> servers = mockClusterServers(mockCluster);
+      List<ServerAndLoad> list = convertToList(servers);
+      LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
+      List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
+      List<ServerAndLoad> balancedCluster = reconcile(list, plans, servers);
+      LOG.info("Mock Balance : " + printMock(balancedCluster));
+      assertClusterAsBalanced(balancedCluster);
+      List<RegionPlan> secondPlans =  loadBalancer.balanceCluster(servers);
+      assertNull(secondPlans);
+      for (Map.Entry<ServerName, List<HRegionInfo>> entry : servers.entrySet()) {
+        returnRegions(entry.getValue());
+        returnServer(entry.getKey());
+      }
+    }
+
+  }
+
+  @Test
+  public void testMoveCost() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    StochasticLoadBalancer.CostFunction
+        costFunction = new StochasticLoadBalancer.MoveCostFunction(conf);
+    for (int[] mockCluster : clusterStateMocks) {
+      BaseLoadBalancer.Cluster cluster = mockCluster(mockCluster);
+      costFunction.init(cluster);
+      double cost = costFunction.cost();
+      assertEquals(0.0f, cost, 0.001);
+
+      // cluster region number is smaller than maxMoves=600
+      cluster.setNumRegions(200);
+      cluster.setNumMovedRegions(10);
+      cost = costFunction.cost();
+      assertEquals(0.05f, cost, 0.001);
+      cluster.setNumMovedRegions(100);
+      cost = costFunction.cost();
+      assertEquals(0.5f, cost, 0.001);
+      cluster.setNumMovedRegions(200);
+      cost = costFunction.cost();
+      assertEquals(1.0f, cost, 0.001);
+
+
+      // cluster region number is bigger than maxMoves=2500
+      cluster.setNumRegions(10000);
+      cluster.setNumMovedRegions(250);
+      cost = costFunction.cost();
+      assertEquals(0.1f, cost, 0.001);
+      cluster.setNumMovedRegions(1250);
+      cost = costFunction.cost();
+      assertEquals(0.5f, cost, 0.001);
+      cluster.setNumMovedRegions(2500);
+      cost = costFunction.cost();
+      assertEquals(1.0f, cost, 0.01);
+    }
+  }
+
+  @Test
+  public void testSkewCost() {
+    Configuration conf = HBaseConfiguration.create();
+    StochasticLoadBalancer.CostFunction
+        costFunction = new StochasticLoadBalancer.RegionCountSkewCostFunction(conf);
+    for (int[] mockCluster : clusterStateMocks) {
+      costFunction.init(mockCluster(mockCluster));
+      double cost = costFunction.cost();
+      assertTrue(cost >= 0);
+      assertTrue(cost <= 1.01);
+    }
+
+    costFunction.init(mockCluster(new int[]{0, 0, 0, 0, 1}));
+    assertEquals(0,costFunction.cost(), 0.01);
+    costFunction.init(mockCluster(new int[]{0, 0, 0, 1, 1}));
+    assertEquals(0, costFunction.cost(), 0.01);
+    costFunction.init(mockCluster(new int[]{0, 0, 1, 1, 1}));
+    assertEquals(0, costFunction.cost(), 0.01);
+    costFunction.init(mockCluster(new int[]{0, 1, 1, 1, 1}));
+    assertEquals(0, costFunction.cost(), 0.01);
+    costFunction.init(mockCluster(new int[]{1, 1, 1, 1, 1}));
+    assertEquals(0, costFunction.cost(), 0.01);
+    costFunction.init(mockCluster(new int[]{10000, 0, 0, 0, 0}));
+    assertEquals(1, costFunction.cost(), 0.01);
+  }
+
+  @Test
+  public void testTableSkewCost() {
+    Configuration conf = HBaseConfiguration.create();
+    StochasticLoadBalancer.CostFunction
+        costFunction = new StochasticLoadBalancer.TableSkewCostFunction(conf);
+    for (int[] mockCluster : clusterStateMocks) {
+      BaseLoadBalancer.Cluster cluster = mockCluster(mockCluster);
+      costFunction.init(cluster);
+      double cost = costFunction.cost();
+      assertTrue(cost >= 0);
+      assertTrue(cost <= 1.01);
+    }
+  }
+
+  @Test
+  public void testCostFromArray() {
+    Configuration conf = HBaseConfiguration.create();
+    StochasticLoadBalancer.CostFromRegionLoadFunction
+        costFunction = new StochasticLoadBalancer.MemstoreSizeCostFunction(conf);
+    costFunction.init(mockCluster(new int[]{0, 0, 0, 0, 1}));
+
+    double[] statOne = new double[100];
+    for (int i =0; i < 100; i++) {
+      statOne[i] = 10;
+    }
+    assertEquals(0, costFunction.costFromArray(statOne), 0.01);
+
+    double[] statTwo= new double[101];
+    for (int i =0; i < 100; i++) {
+      statTwo[i] = 0;
+    }
+    statTwo[100] = 100;
+    assertEquals(1, costFunction.costFromArray(statTwo), 0.01);
+
+    double[] statThree = new double[200];
+    for (int i =0; i < 100; i++) {
+      statThree[i] = (0);
+      statThree[i+100] = 100;
+    }
+    assertEquals(0.5, costFunction.costFromArray(statThree), 0.01);
+  }
+
+  @Test(timeout =  60000)
+  public void testLosingRs() throws Exception {
+    int numNodes = 3;
+    int numRegions = 20;
+    int numRegionsPerServer = 3; //all servers except one
+    int replication = 1;
+    int numTables = 2;
+
+    Map<ServerName, List<HRegionInfo>> serverMap =
+        createServerMap(numNodes, numRegions, numRegionsPerServer, replication, numTables);
+    List<ServerAndLoad> list = convertToList(serverMap);
+
+
+    List<RegionPlan> plans = loadBalancer.balanceCluster(serverMap);
+    assertNotNull(plans);
+
+    // Apply the plan to the mock cluster.
+    List<ServerAndLoad> balancedCluster = reconcile(list, plans, serverMap);
+
+    assertClusterAsBalanced(balancedCluster);
+
+    ServerName sn = serverMap.keySet().toArray(new ServerName[serverMap.size()])[0];
+
+    ServerName deadSn = ServerName.valueOf(sn.getHostname(), sn.getPort(), sn.getStartcode() - 100);
+
+    serverMap.put(deadSn, new ArrayList<HRegionInfo>(0));
+
+    plans = loadBalancer.balanceCluster(serverMap);
+    assertNull(plans);
+  }
+
+  @Test
+  public void testReplicaCost() {
+    Configuration conf = HBaseConfiguration.create();
+    StochasticLoadBalancer.CostFunction
+        costFunction = new StochasticLoadBalancer.RegionReplicaHostCostFunction(conf);
+    for (int[] mockCluster : clusterStateMocks) {
+      BaseLoadBalancer.Cluster cluster = mockCluster(mockCluster);
+      costFunction.init(cluster);
+      double cost = costFunction.cost();
+      assertTrue(cost >= 0);
+      assertTrue(cost <= 1.01);
+    }
+  }
+
+  @Test
+  public void testReplicaCostForReplicas() {
+    Configuration conf = HBaseConfiguration.create();
+    StochasticLoadBalancer.CostFunction
+        costFunction = new StochasticLoadBalancer.RegionReplicaHostCostFunction(conf);
+
+    int [] servers = new int[] {3,3,3,3,3};
+    TreeMap<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(servers);
+
+    BaseLoadBalancer.Cluster cluster;
+
+    cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
+    costFunction.init(cluster);
+    double costWithoutReplicas = costFunction.cost();
+    assertEquals(0, costWithoutReplicas, 0);
+
+    // replicate the region from first server to the last server
+    HRegionInfo replica1 = RegionReplicaUtil.getRegionInfoForReplica(
+      clusterState.firstEntry().getValue().get(0),1);
+    clusterState.lastEntry().getValue().add(replica1);
+
+    cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
+    costFunction.init(cluster);
+    double costWith1ReplicaDifferentServer = costFunction.cost();
+
+    assertEquals(0, costWith1ReplicaDifferentServer, 0);
+
+    // add a third replica to the last server
+    HRegionInfo replica2 = RegionReplicaUtil.getRegionInfoForReplica(replica1, 2);
+    clusterState.lastEntry().getValue().add(replica2);
+
+    cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
+    costFunction.init(cluster);
+    double costWith1ReplicaSameServer = costFunction.cost();
+
+    assertTrue(costWith1ReplicaDifferentServer < costWith1ReplicaSameServer);
+
+    // test with replication = 4 for following:
+
+    HRegionInfo replica3;
+    Iterator<Entry<ServerName, List<HRegionInfo>>> it;
+    Entry<ServerName, List<HRegionInfo>> entry;
+
+    clusterState = mockClusterServers(servers);
+    it = clusterState.entrySet().iterator();
+    entry = it.next(); //first server
+    HRegionInfo hri = entry.getValue().get(0);
+    replica1 = RegionReplicaUtil.getRegionInfoForReplica(hri, 1);
+    replica2 = RegionReplicaUtil.getRegionInfoForReplica(hri, 2);
+    replica3 = RegionReplicaUtil.getRegionInfoForReplica(hri, 3);
+    entry.getValue().add(replica1);
+    entry.getValue().add(replica2);
+    it.next().getValue().add(replica3); //2nd server
+
+    cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
+    costFunction.init(cluster);
+    double costWith3ReplicasSameServer = costFunction.cost();
+
+    clusterState = mockClusterServers(servers);
+    hri = clusterState.firstEntry().getValue().get(0);
+    replica1 = RegionReplicaUtil.getRegionInfoForReplica(hri, 1);
+    replica2 = RegionReplicaUtil.getRegionInfoForReplica(hri, 2);
+    replica3 = RegionReplicaUtil.getRegionInfoForReplica(hri, 3);
+
+    clusterState.firstEntry().getValue().add(replica1);
+    clusterState.lastEntry().getValue().add(replica2);
+    clusterState.lastEntry().getValue().add(replica3);
+
+    cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
+    costFunction.init(cluster);
+    double costWith2ReplicasOnTwoServers = costFunction.cost();
+
+    assertTrue(costWith2ReplicasOnTwoServers < costWith3ReplicasSameServer);
+  }
+
+  @Test
+  public void testNeedsBalanceForColocatedReplicas() {
+    // check for the case where there are two hosts and with one rack, and where
+    // both the replicas are hosted on the same server
+    List<HRegionInfo> regions = randomRegions(1);
+    ServerName s1 = ServerName.valueOf("host1", 1000, 11111);
+    ServerName s2 = ServerName.valueOf("host11", 1000, 11111);
+    Map<ServerName, List<HRegionInfo>> map = new HashMap<ServerName, List<HRegionInfo>>();
+    map.put(s1, regions);
+    regions.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
+    // until the step above s1 holds two replicas of a region
+    regions = randomRegions(1);
+    map.put(s2, regions);
+    assertTrue(loadBalancer.needsBalance(new Cluster(map, null, null, null)));
+    // check for the case where there are two hosts on the same rack and there are two racks
+    // and both the replicas are on the same rack
+    map.clear();
+    regions = randomRegions(1);
+    List<HRegionInfo> regionsOnS2 = new ArrayList<HRegionInfo>(1);
+    regionsOnS2.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
+    map.put(s1, regions);
+    map.put(s2, regionsOnS2);
+    // add another server so that the cluster has some host on another rack
+    map.put(ServerName.valueOf("host2", 1000, 11111), randomRegions(1));
+    assertTrue(loadBalancer.needsBalance(new Cluster(map, null, null,
+        new ForTestRackManagerOne())));
+  }
+
+  @Test (timeout = 60000)
+  public void testSmallCluster() {
+    int numNodes = 10;
+    int numRegions = 1000;
+    int numRegionsPerServer = 40; //all servers except one
+    int replication = 1;
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+
+  @Test (timeout = 60000)
+  public void testSmallCluster2() {
+    int numNodes = 20;
+    int numRegions = 2000;
+    int numRegionsPerServer = 40; //all servers except one
+    int replication = 1;
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+
+  @Test (timeout = 60000)
+  public void testSmallCluster3() {
+    int numNodes = 20;
+    int numRegions = 2000;
+    int numRegionsPerServer = 1; // all servers except one
+    int replication = 1;
+    int numTables = 10;
+    /* fails because of max moves */
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, false);
+  }
+
+  @Test (timeout = 800000)
+  public void testMidCluster() {
+    int numNodes = 100;
+    int numRegions = 10000;
+    int numRegionsPerServer = 60; // all servers except one
+    int replication = 1;
+    int numTables = 40;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+
+  @Test (timeout = 800000)
+  public void testMidCluster2() {
+    int numNodes = 200;
+    int numRegions = 100000;
+    int numRegionsPerServer = 40; // all servers except one
+    int replication = 1;
+    int numTables = 400;
+    testWithCluster(numNodes,
+        numRegions,
+        numRegionsPerServer,
+        replication,
+        numTables,
+        false, /* num large num regions means may not always get to best balance with one run */
+        false);
+  }
+
+
+  @Test (timeout = 800000)
+  public void testMidCluster3() {
+    int numNodes = 100;
+    int numRegions = 2000;
+    int numRegionsPerServer = 9; // all servers except one
+    int replication = 1;
+    int numTables = 110;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+    // TODO(eclark): Make sure that the tables are well distributed.
+  }
+
+  @Test
+  public void testLargeCluster() {
+    int numNodes = 1000;
+    int numRegions = 100000; //100 regions per RS
+    int numRegionsPerServer = 80; //all servers except one
+    int numTables = 100;
+    int replication = 1;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+
+  @Test (timeout = 800000)
+  public void testRegionReplicasOnSmallCluster() {
+    int numNodes = 10;
+    int numRegions = 1000;
+    int replication = 3; // 3 replicas per region
+    int numRegionsPerServer = 80; //all regions are mostly balanced
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+
+  @Test (timeout = 800000)
+  public void testRegionReplicationOnMidClusterSameHosts() {
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
+    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+    loadBalancer.setConf(conf);
+    int numHosts = 100;
+    int numRegions = 100 * 100;
+    int replication = 3; // 3 replicas per region
+    int numRegionsPerServer = 5;
+    int numTables = 10;
+    Map<ServerName, List<HRegionInfo>> serverMap =
+        createServerMap(numHosts, numRegions, numRegionsPerServer, replication, numTables);
+    int numNodesPerHost = 4;
+
+    // create a new map with 4 RS per host.
+    Map<ServerName, List<HRegionInfo>> newServerMap = new TreeMap<ServerName, List<HRegionInfo>>(serverMap);
+    for (Map.Entry<ServerName, List<HRegionInfo>> entry : serverMap.entrySet()) {
+      for (int i=1; i < numNodesPerHost; i++) {
+        ServerName s1 = entry.getKey();
+        ServerName s2 = ServerName.valueOf(s1.getHostname(), s1.getPort() + i, 1); // create an RS for the same host
+        newServerMap.put(s2, new ArrayList<HRegionInfo>());
+      }
+    }
+
+    testWithCluster(newServerMap, null, true, true);
+  }
+
+  private static class ForTestRackManager extends RackManager {
+    int numRacks;
+    public ForTestRackManager(int numRacks) {
+      this.numRacks = numRacks;
+    }
+    @Override
+    public String getRack(ServerName server) {
+      return "rack_" + (server.hashCode() % numRacks);
+    }
+  }
+
+  private static class ForTestRackManagerOne extends RackManager {
+  @Override
+    public String getRack(ServerName server) {
+      return server.getHostname().endsWith("1") ? "rack1" : "rack2";
+    }
+  }
+
+  @Test (timeout = 800000)
+  public void testRegionReplicationOnMidClusterWithRacks() {
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 10000000L);
+    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
+    loadBalancer.setConf(conf);
+    int numNodes = 30;
+    int numRegions = numNodes * 30;
+    int replication = 3; // 3 replicas per region
+    int numRegionsPerServer = 28;
+    int numTables = 10;
+    int numRacks = 4; // all replicas should be on a different rack
+    Map<ServerName, List<HRegionInfo>> serverMap =
+        createServerMap(numNodes, numRegions, numRegionsPerServer, replication, numTables);
+    RackManager rm = new ForTestRackManager(numRacks);
+
+    testWithCluster(serverMap, rm, false, true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2fba25b6/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java
new file mode 100644
index 0000000..976ba45
--- /dev/null
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestShell.java
@@ -0,0 +1,39 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.jruby.embed.PathType;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, LargeTests.class })
+public class TestShell extends AbstractTestShell {
+
+  @Test
+  public void testRunShellTests() throws IOException {
+    System.setProperty("shell.test.exclude", "replication_admin_test.rb");
+    // Start all ruby tests
+    jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb");
+  }
+
+}


[20/50] [abbrv] hbase git commit: HBASE-14888 ClusterSchema: Add Namespace Operations

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 8e51f25..4472b65 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
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -79,6 +78,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.executor.ExecutorType;
@@ -97,17 +97,14 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
 import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
-import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
-import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
-import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@@ -185,7 +182,7 @@ import com.google.protobuf.Service;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HMaster extends HRegionServer implements MasterServices, Server {
+public class HMaster extends HRegionServer implements MasterServices {
   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
 
   /**
@@ -256,8 +253,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   // Tracker for region normalizer state
   private RegionNormalizerTracker regionNormalizerTracker;
 
-  /** Namespace stuff */
-  private TableNamespaceManager tableNamespaceManager;
+  private ClusterSchemaService clusterSchemaService;
 
   // Metrics for the HMaster
   final MetricsMaster metricsMaster;
@@ -368,9 +364,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * Remaining steps of initialization occur in
    * #finishActiveMasterInitialization(MonitoredTask) after
    * the master becomes the active one.
-   *
-   * @throws KeeperException
-   * @throws IOException
    */
   public HMaster(final Configuration conf, CoordinatedStateManager csm)
       throws IOException, KeeperException {
@@ -570,10 +563,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * Initialize all ZK based system trackers.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws CoordinatedStateException
    */
   void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException, CoordinatedStateException {
@@ -588,12 +577,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       this.balancer, this.service, this.metricsMaster,
       this.tableLockManager, tableStateManager);
 
-    this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
-        this.serverManager);
+    this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
     this.regionServerTracker.start();
 
-    this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this,
-      this.serverManager);
+    this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager);
     this.drainingServerTracker.start();
 
     // Set the cluster as up.  If new RSs, they'll be waiting on this before
@@ -630,11 +617,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * <li>Ensure assignment of meta/namespace regions<li>
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
-   *
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws CoordinatedStateException
    */
   private void finishActiveMasterInitialization(MonitoredTask status)
       throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
@@ -781,8 +763,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     this.catalogJanitorChore = new CatalogJanitor(this, this);
     getChoreService().scheduleChore(catalogJanitorChore);
 
-    status.setStatus("Starting namespace manager");
-    initNamespace();
+    status.setStatus("Starting cluster schema service");
+    initClusterSchemaService();
 
     if (this.cpHost != null) {
       try {
@@ -848,11 +830,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * Create a {@link ServerManager} instance.
-   * @param master
-   * @param services
-   * @return An instance of {@link ServerManager}
-   * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
-   * @throws IOException
    */
   ServerManager createServerManager(final Server master,
       final MasterServices services)
@@ -874,7 +851,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
           RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId);
           LOG.info("Closing excess replica of meta region " + r.getRegion());
           // send a close and wait for a max of 30 seconds
-          ServerManager.closeRegionSilentlyAndWait(getConnection(), r.getServerName(),
+          ServerManager.closeRegionSilentlyAndWait(getClusterConnection(), r.getServerName(),
               r.getRegion(), 30000);
           ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId));
         }
@@ -888,12 +865,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * Check <code>hbase:meta</code> is assigned. If not, assign it.
-   * @param status MonitoredTask
-   * @param previouslyFailedMetaRSs
-   * @param replicaId
-   * @throws InterruptedException
-   * @throws IOException
-   * @throws KeeperException
    */
   void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
       throws InterruptedException, IOException, KeeperException {
@@ -915,7 +886,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         metaState.getServerName(), null);
 
     if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation(
-        this.getConnection(), this.getZooKeeper(), timeout, replicaId)) {
+        this.getClusterConnection(), this.getZooKeeper(), timeout, replicaId)) {
       ServerName currentMetaServer = metaState.getServerName();
       if (serverManager.isServerOnline(currentMetaServer)) {
         if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
@@ -965,10 +936,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     status.setStatus("META assigned.");
   }
 
-  void initNamespace() throws IOException {
-    //create namespace manager
-    tableNamespaceManager = new TableNamespaceManager(this);
-    tableNamespaceManager.start();
+  void initClusterSchemaService() throws IOException, InterruptedException {
+    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
+    this.clusterSchemaService.startAndWait();
+    if (!this.clusterSchemaService.isRunning()) throw new HBaseIOException("Failed start");
   }
 
   void initQuotaManager() throws IOException {
@@ -1014,7 +985,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   /**
    * This function returns a set of region server names under hbase:meta recovering region ZK node
    * @return Set of meta server names which were recorded in ZK
-   * @throws KeeperException
    */
   private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
     Set<ServerName> result = new HashSet<ServerName>();
@@ -1050,11 +1020,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return tableStateManager;
   }
 
-  @Override
-  public TableNamespaceManager getTableNamespaceManager() {
-    return tableNamespaceManager;
-  }
-
   /*
    * Start up all services. If any of these threads gets an unhandled exception
    * then they just die with a logged message.  This should be fine because
@@ -1201,7 +1166,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * @return Get remote side's InetAddress
-   * @throws UnknownHostException
    */
   InetAddress getRemoteInetAddress(final int port,
       final long serverStartCode) throws UnknownHostException {
@@ -1336,9 +1300,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * Perform normalization of cluster (invoked by {@link RegionNormalizerChore}).
    *
    * @return true if normalization step was performed successfully, false otherwise
-   *   (specifically, if HMaster hasn't been initialized properly or normalization
-   *   is globally disabled)
-   * @throws IOException
+   *    (specifically, if HMaster hasn't been initialized properly or normalization
+   *    is globally disabled)
    */
   public boolean normalizeRegions() throws IOException {
     if (!this.initialized) {
@@ -1478,9 +1441,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     if (isStopped()) {
       throw new MasterNotRunningException();
     }
-
+    checkInitialized();
     String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
-    ensureNamespaceExists(namespace);
+    this.clusterSchemaService.getNamespace(namespace);
 
     HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
@@ -2167,8 +2130,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * The set of loaded coprocessors is stored in a static set. Since it's
    * statically allocated, it does not require that HMaster's cpHost be
    * initialized prior to accessing it.
-   * @return a String representation of the set of names of the loaded
-   * coprocessors.
+   * @return a String representation of the set of names of the loaded coprocessors.
    */
   public static String getLoadedCoprocessors() {
     return CoprocessorHost.getLoadedCoprocessors().toString();
@@ -2305,18 +2267,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
     checkServiceStarted();
-    if (!this.initialized) {
-      throw new PleaseHoldException("Master is initializing");
-    }
+    if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
   }
 
-  void checkNamespaceManagerReady() throws IOException {
-    checkInitialized();
-    if (tableNamespaceManager == null ||
-        !tableNamespaceManager.isTableAvailableAndInitialized(true)) {
-      throw new IOException("Table Namespace Manager not ready yet, try again later");
-    }
-  }
   /**
    * Report whether this master is currently the active master or not.
    * If not active master, we are parked on ZK waiting to become active.
@@ -2411,7 +2364,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   /**
    * Utility for constructing an instance of the passed HMaster class.
    * @param masterClass
-   * @param conf
    * @return HMaster instance.
    */
   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
@@ -2452,138 +2404,116 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
-  public void createNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
-    checkNamespaceManagerReady();
-    if (cpHost != null) {
-      if (cpHost.preCreateNamespace(descriptor)) {
-        return;
-      }
-    }
-    createNamespaceSync(descriptor, nonceGroup, nonce);
-    if (cpHost != null) {
-      cpHost.postCreateNamespace(descriptor);
-    }
+  public ClusterSchema getClusterSchema() {
+    return this.clusterSchemaService;
   }
 
-  @Override
-  public void createNamespaceSync(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
+  /**
+   * Create a new Namespace.
+   * @param namespaceDescriptor descriptor for new Namespace
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return procedure id
+   */
+  long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
+      final long nonce)
+  throws IOException {
+    checkInitialized();
+    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
+    if (this.cpHost != null && this.cpHost.preCreateNamespace(namespaceDescriptor)) {
+      throw new BypassCoprocessorException();
+    }
+    LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new CreateNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    long procId = getClusterSchema().createNamespace(namespaceDescriptor, nonceGroup, nonce);
+    if (this.cpHost != null) this.cpHost.postCreateNamespace(namespaceDescriptor);
+    return procId;
   }
 
-  @Override
-  public void modifyNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
-    checkNamespaceManagerReady();
-    if (cpHost != null) {
-      if (cpHost.preModifyNamespace(descriptor)) {
-        return;
-      }
+  /**
+   * Modify an existing Namespace.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return procedure id
+   */
+  long modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
+      final long nonce)
+  throws IOException {
+    checkInitialized();
+    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
+    if (this.cpHost != null && this.cpHost.preModifyNamespace(namespaceDescriptor)) {
+      throw new BypassCoprocessorException();
     }
-    LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
+    LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new ModifyNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
-    if (cpHost != null) {
-      cpHost.postModifyNamespace(descriptor);
-    }
+    long procId = getClusterSchema().modifyNamespace(namespaceDescriptor, nonceGroup, nonce);
+    if (this.cpHost != null) this.cpHost.postModifyNamespace(namespaceDescriptor);
+    return procId;
   }
 
-  @Override
-  public void deleteNamespace(
-      final String name,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    checkNamespaceManagerReady();
-    if (cpHost != null) {
-      if (cpHost.preDeleteNamespace(name)) {
-        return;
-      }
+  /**
+   * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return procedure id
+   */
+  long deleteNamespace(final String name, final long nonceGroup, final long nonce)
+  throws IOException {
+    checkInitialized();
+    if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) {
+      throw new BypassCoprocessorException();
     }
     LOG.info(getClientIdAuditPrefix() + " delete " + name);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
-    if (cpHost != null) {
-      cpHost.postDeleteNamespace(name);
-    }
+    long procId = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
+    if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
+    return procId;
   }
 
   /**
-   * Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException
-   *
-   * @param name the namespace to check
-   * @throws IOException if the namespace manager is not ready yet.
-   * @throws NamespaceNotFoundException if the namespace does not exists
+   * Get a Namespace
+   * @param name Name of the Namespace
+   * @return Namespace descriptor for <code>name</code>
    */
-  private void ensureNamespaceExists(final String name)
-      throws IOException, NamespaceNotFoundException {
-    checkNamespaceManagerReady();
-    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
-    if (nsd == null) {
-      throw new NamespaceNotFoundException(name);
-    }
+  NamespaceDescriptor getNamespace(String name) throws IOException {
+    checkInitialized();
+    if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
+    NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
+    if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
+    return nsd;
   }
 
-  @Override
-  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
-    checkNamespaceManagerReady();
-
+  /**
+   * Get all Namespaces
+   * @return All Namespace descriptors
+   */
+  List<NamespaceDescriptor> getNamespaces() throws IOException {
+    checkInitialized();
+    final List<NamespaceDescriptor> nsds = new ArrayList<NamespaceDescriptor>();
+    boolean bypass = false;
     if (cpHost != null) {
-      cpHost.preGetNamespaceDescriptor(name);
+      bypass = cpHost.preListNamespaceDescriptors(nsds);
     }
-
-    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
-    if (nsd == null) {
-      throw new NamespaceNotFoundException(name);
-    }
-
-    if (cpHost != null) {
-      cpHost.postGetNamespaceDescriptor(nsd);
+    if (!bypass) {
+      nsds.addAll(this.clusterSchemaService.getNamespaces());
+      if (this.cpHost != null) this.cpHost.postListNamespaceDescriptors(nsds);
     }
-
-    return nsd;
+    return nsds;
   }
 
   @Override
-  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
-    checkNamespaceManagerReady();
-
-    final List<NamespaceDescriptor> descriptors = new ArrayList<NamespaceDescriptor>();
-    boolean bypass = false;
-    if (cpHost != null) {
-      bypass = cpHost.preListNamespaceDescriptors(descriptors);
-    }
-
-    if (!bypass) {
-      descriptors.addAll(tableNamespaceManager.list());
+  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
+    checkInitialized();
+    return listTableNames(name, null, true);
+  }
 
-      if (cpHost != null) {
-        cpHost.postListNamespaceDescriptors(descriptors);
-      }
-    }
-    return descriptors;
+  @Override
+  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+    checkInitialized();
+    return listTableDescriptors(name, null, null, true);
   }
 
   @Override
@@ -2617,21 +2547,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return procInfoList;
   }
 
-  @Override
-  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
-    ensureNamespaceExists(name);
-    return listTableDescriptors(name, null, null, true);
-  }
-
-  @Override
-  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
-    ensureNamespaceExists(name);
-    return listTableNames(name, null, true);
-  }
-
   /**
    * Returns the list of table descriptors that match the specified request
-   *
    * @param namespace the namespace to query, or null if querying for all
    * @param regex The regular expression to match against, or null if querying for all
    * @param tableNameList the list of table names, or null if querying for all
@@ -2640,51 +2557,17 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    */
   public List<HTableDescriptor> listTableDescriptors(final String namespace, final String regex,
       final List<TableName> tableNameList, final boolean includeSysTables)
-      throws IOException {
-    final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
-
-    boolean bypass = false;
-    if (cpHost != null) {
-      bypass = cpHost.preGetTableDescriptors(tableNameList, descriptors, regex);
-    }
-
+  throws IOException {
+    List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
+    boolean bypass = cpHost != null?
+        cpHost.preGetTableDescriptors(tableNameList, htds, regex): false;
     if (!bypass) {
-      if (tableNameList == null || tableNameList.size() == 0) {
-        // request for all TableDescriptors
-        Collection<HTableDescriptor> htds;
-        if (namespace != null && namespace.length() > 0) {
-          htds = tableDescriptors.getByNamespace(namespace).values();
-        } else {
-          htds = tableDescriptors.getAll().values();
-        }
-
-        for (HTableDescriptor desc: htds) {
-          if (tableStateManager.isTablePresent(desc.getTableName())
-              && (includeSysTables || !desc.getTableName().isSystemTable())) {
-            descriptors.add(desc);
-          }
-        }
-      } else {
-        for (TableName s: tableNameList) {
-          if (tableStateManager.isTablePresent(s)) {
-            HTableDescriptor desc = tableDescriptors.get(s);
-            if (desc != null) {
-              descriptors.add(desc);
-            }
-          }
-        }
-      }
-
-      // Retains only those matched by regular expression.
-      if (regex != null) {
-        filterTablesByRegex(descriptors, Pattern.compile(regex));
-      }
-
+      htds = getTableDescriptors(htds, namespace, regex, tableNameList, includeSysTables);
       if (cpHost != null) {
-        cpHost.postGetTableDescriptors(tableNameList, descriptors, regex);
+        cpHost.postGetTableDescriptors(tableNameList, htds, regex);
       }
     }
-    return descriptors;
+    return htds;
   }
 
   /**
@@ -2696,46 +2579,58 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    */
   public List<TableName> listTableNames(final String namespace, final String regex,
       final boolean includeSysTables) throws IOException {
-    final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
-
-    boolean bypass = false;
-    if (cpHost != null) {
-      bypass = cpHost.preGetTableNames(descriptors, regex);
+    List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
+    boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): false;
+    if (!bypass) {
+      htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables);
+      if (cpHost != null) cpHost.postGetTableNames(htds, regex);
     }
+    List<TableName> result = new ArrayList<TableName>(htds.size());
+    for (HTableDescriptor htd: htds) result.add(htd.getTableName());
+    return result;
+  }
 
-    if (!bypass) {
-      // get all descriptors
-      Collection<HTableDescriptor> htds;
+  /**
+   * @return list of table table descriptors after filtering by regex and whether to include system
+   *    tables, etc.
+   * @throws IOException
+   */
+  private List<HTableDescriptor> getTableDescriptors(final List<HTableDescriptor> htds,
+      final String namespace, final String regex, final List<TableName> tableNameList,
+      final boolean includeSysTables)
+  throws IOException {
+    if (tableNameList == null || tableNameList.size() == 0) {
+      // request for all TableDescriptors
+      Collection<HTableDescriptor> allHtds;
       if (namespace != null && namespace.length() > 0) {
-        htds = tableDescriptors.getByNamespace(namespace).values();
+        // Do a check on the namespace existence. Will fail if does not exist.
+        this.clusterSchemaService.getNamespace(namespace);
+        allHtds = tableDescriptors.getByNamespace(namespace).values();
       } else {
-        htds = tableDescriptors.getAll().values();
+        allHtds = tableDescriptors.getAll().values();
       }
-
-      for (HTableDescriptor htd: htds) {
-        if (includeSysTables || !htd.getTableName().isSystemTable()) {
-          descriptors.add(htd);
+      for (HTableDescriptor desc: allHtds) {
+        if (tableStateManager.isTablePresent(desc.getTableName())
+            && (includeSysTables || !desc.getTableName().isSystemTable())) {
+          htds.add(desc);
         }
       }
-
-      // Retains only those matched by regular expression.
-      if (regex != null) {
-        filterTablesByRegex(descriptors, Pattern.compile(regex));
-      }
-
-      if (cpHost != null) {
-        cpHost.postGetTableNames(descriptors, regex);
+    } else {
+      for (TableName s: tableNameList) {
+        if (tableStateManager.isTablePresent(s)) {
+          HTableDescriptor desc = tableDescriptors.get(s);
+          if (desc != null) {
+            htds.add(desc);
+          }
+        }
       }
     }
 
-    List<TableName> result = new ArrayList<TableName>(descriptors.size());
-    for (HTableDescriptor htd: descriptors) {
-      result.add(htd.getTableName());
-    }
-    return result;
+    // Retains only those matched by regular expression.
+    if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex));
+    return htds;
   }
 
-
   /**
    * Removes the table descriptors that don't match the pattern.
    * @param descriptors list of table descriptors to filter
@@ -2848,11 +2743,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * Queries the state of the {@link RegionNormalizerTracker}. If it's not initialized,
    * false is returned.
    */
-   public boolean isNormalizerOn() {
-    if (null == regionNormalizerTracker) {
-      return false;
-    }
-    return regionNormalizerTracker.isNormalizerOn();
+  public boolean isNormalizerOn() {
+    return null == regionNormalizerTracker? false: regionNormalizerTracker.isNormalizerOn();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index b269c3d..141fa88 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -55,121 +55,21 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
-import org.apache.hadoop.hbase.protobuf.generated.*;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -188,8 +88,8 @@ import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
 
@@ -458,11 +358,11 @@ public class MasterRpcServices extends RSRpcServices
   public CreateNamespaceResponse createNamespace(RpcController controller,
      CreateNamespaceRequest request) throws ServiceException {
     try {
-      master.createNamespace(
+      long procId = master.createNamespace(
         ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
         request.getNonceGroup(),
         request.getNonce());
-      return CreateNamespaceResponse.getDefaultInstance();
+      return CreateNamespaceResponse.newBuilder().setProcId(procId).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -506,11 +406,11 @@ public class MasterRpcServices extends RSRpcServices
   public DeleteNamespaceResponse deleteNamespace(RpcController controller,
       DeleteNamespaceRequest request) throws ServiceException {
     try {
-      master.deleteNamespace(
+      long procId = master.deleteNamespace(
         request.getNamespaceName(),
         request.getNonceGroup(),
         request.getNonce());
-      return DeleteNamespaceResponse.getDefaultInstance();
+      return DeleteNamespaceResponse.newBuilder().setProcId(procId).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -832,7 +732,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       return GetNamespaceDescriptorResponse.newBuilder()
         .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
-            master.getNamespaceDescriptor(request.getNamespaceName())))
+            master.getNamespace(request.getNamespaceName())))
         .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -977,10 +877,8 @@ public class MasterRpcServices extends RSRpcServices
 
   /**
    * Checks if the specified procedure is done.
-   * @return true if the procedure is done,
-   *   false if the procedure is in the process of completing
-   * @throws ServiceException if invalid procedure, or
-   *  a failed procedure with progress failure reason.
+   * @return true if the procedure is done, false if the procedure is in the process of completing
+   * @throws ServiceException if invalid procedure or failed procedure with progress failure reason.
    */
   @Override
   public IsProcedureDoneResponse isProcedureDone(RpcController controller,
@@ -1120,7 +1018,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       ListNamespaceDescriptorsResponse.Builder response =
         ListNamespaceDescriptorsResponse.newBuilder();
-      for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) {
+      for(NamespaceDescriptor ns: master.getNamespaces()) {
         response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
       }
       return response.build();
@@ -1200,11 +1098,11 @@ public class MasterRpcServices extends RSRpcServices
   public ModifyNamespaceResponse modifyNamespace(RpcController controller,
       ModifyNamespaceRequest request) throws ServiceException {
     try {
-      master.modifyNamespace(
+      long procId = master.modifyNamespace(
         ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
         request.getNonceGroup(),
         request.getNonce());
-      return ModifyNamespaceResponse.getDefaultInstance();
+      return ModifyNamespaceResponse.newBuilder().setProcId(procId).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1305,10 +1203,9 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       master.snapshotManager.checkSnapshotSupport();
 
-    // ensure namespace exists
+      // Ensure namespace exists. Will throw exception if non-known NS.
       TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
-      master.getNamespaceDescriptor(dstTable.getNamespaceAsString());
-
+      master.getNamespace(dstTable.getNamespaceAsString());
       SnapshotDescription reqSnapshot = request.getSnapshot();
       master.snapshotManager.restoreSnapshot(reqSnapshot);
       return RestoreSnapshotResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index af0e490..ec7db0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,21 +21,20 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 
 import com.google.protobuf.Service;
@@ -46,6 +45,11 @@ import com.google.protobuf.Service;
 @InterfaceAudience.Private
 public interface MasterServices extends Server {
   /**
+   * @return Master's instance of {@link ClusterSchema}
+   */
+  ClusterSchema getClusterSchema();
+
+  /**
    * @return Master's instance of the {@link AssignmentManager}
    */
   AssignmentManager getAssignmentManager();
@@ -81,11 +85,6 @@ public interface MasterServices extends Server {
   MasterCoprocessorHost getMasterCoprocessorHost();
 
   /**
-   * @return Master's instance of {@link TableNamespaceManager}
-   */
-  TableNamespaceManager getTableNamespaceManager();
-
-  /**
    * @return Master's instance of {@link MasterQuotaManager}
    */
   MasterQuotaManager getMasterQuotaManager();
@@ -280,54 +279,6 @@ public interface MasterServices extends Server {
   boolean isInitialized();
 
   /**
-   * Create a new namespace
-   * @param descriptor descriptor which describes the new namespace
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void createNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
-   * Create a new namespace synchronously.
-   * @param descriptor descriptor which describes the new namespace
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void createNamespaceSync(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
-   * Modify an existing namespace
-   * @param descriptor descriptor which updates the existing namespace
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void modifyNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
-   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
-   * @param name namespace name
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void deleteNamespace(
-      final String name,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
    * Abort a procedure.
    * @param procId ID of the procedure
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
@@ -338,21 +289,6 @@ public interface MasterServices extends Server {
       throws IOException;
 
   /**
-   * Get a namespace descriptor by name
-   * @param name name of namespace descriptor
-   * @return A descriptor
-   * @throws IOException
-   */
-  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException;
-
-  /**
-   * List available namespace descriptors
-   * @return A descriptor
-   * @throws IOException
-   */
-  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException;
-
-  /**
    * List procedures
    * @return procedure list
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 50f07c1..a95279c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -215,7 +215,7 @@ public class ServerManager {
     Configuration c = master.getConfiguration();
     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
-    this.connection = connect ? master.getConnection() : null;
+    this.connection = connect ? master.getClusterConnection() : null;
     int pingMaxAttempts = Math.max(1, master.getConfiguration().getInt(
       "hbase.master.maximum.ping.server.attempts", 10));
     int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index bbeaf76..69d1280 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -27,17 +27,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.commons.lang.StringUtils;
 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.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZKNamespaceManager;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+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;
@@ -46,20 +46,25 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Threads;
 
 import com.google.common.collect.Sets;
 
 /**
- * This is a helper class used to manage the namespace
- * metadata that is stored in TableName.NAMESPACE_TABLE_NAME
- * It also mirrors updates to the ZK store by forwarding updates to
- * {@link org.apache.hadoop.hbase.ZKNamespaceManager}
+ * This is a helper class used internally to manage the namespace metadata that is stored in
+ * TableName.NAMESPACE_TABLE_NAME. It also mirrors updates to the ZK store by forwarding updates to
+ * {@link org.apache.hadoop.hbase.ZKNamespaceManager}.
+ * 
+ * WARNING: Do not use. Go via the higher-level {@link ClusterSchema} API instead. This manager
+ * is likely to go aways anyways.
  */
 @InterfaceAudience.Private
 public class TableNamespaceManager {
@@ -90,7 +95,7 @@ public class TableNamespaceManager {
   private long exclusiveLockTimeoutMs;
   private long sharedLockTimeoutMs;
 
-  public TableNamespaceManager(MasterServices masterServices) {
+  TableNamespaceManager(MasterServices masterServices) {
     this.masterServices = masterServices;
     this.conf = masterServices.getConfiguration();
 
@@ -104,7 +109,7 @@ public class TableNamespaceManager {
 
   public void start() throws IOException {
     if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
-      TableName.NAMESPACE_TABLE_NAME)) {
+        TableName.NAMESPACE_TABLE_NAME)) {
       LOG.info("Namespace table not found. Creating...");
       createNamespaceTable(masterServices);
     }
@@ -113,7 +118,7 @@ public class TableNamespaceManager {
       // Wait for the namespace table to be initialized.
       long startTime = EnvironmentEdgeManager.currentTime();
       int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
-      while (!isTableAvailableAndInitialized(false)) {
+      while (!isTableAvailableAndInitialized()) {
         if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
           // We can't do anything if ns is not online.
           throw new IOException("Timedout " + timeout + "ms waiting for namespace table to "
@@ -269,16 +274,48 @@ public class TableNamespaceManager {
   }
 
   /**
+   * Create Namespace in a blocking manner. Keeps trying until
+   * {@link ClusterSchema.HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY} expires.
+   * Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
+   */
+  private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor)
+  throws IOException {
+    ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
+    long procId =
+      clusterSchema.createNamespace(namespaceDescriptor, HConstants.NO_NONCE, HConstants.NO_NONCE);
+    block(this.masterServices, procId);
+  }
+
+
+  /**
+   * An ugly utility to be removed when refactor TableNamespaceManager.
+   * @throws TimeoutIOException
+   */
+  private static void block(final MasterServices services, final long procId)
+  throws TimeoutIOException {
+    int timeoutInMillis = services.getConfiguration().
+        getInt(ClusterSchema.HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY,
+            ClusterSchema.DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT);
+    long deadlineTs = EnvironmentEdgeManager.currentTime() + timeoutInMillis;
+    ProcedureExecutor<MasterProcedureEnv> procedureExecutor =
+        services.getMasterProcedureExecutor();
+    while(EnvironmentEdgeManager.currentTime() < deadlineTs) {
+      if (procedureExecutor.isFinished(procId)) return;
+      // Sleep some
+      Threads.sleep(10);
+    }
+    throw new TimeoutIOException("Procedure " + procId + " is still running");
+  }
+
+  /**
    * This method checks if the namespace table is assigned and then
-   * tries to create its HTable. If it was already created before, it also makes
+   * tries to create its Table reference. If it was already created before, it also makes
    * sure that the connection isn't closed.
-   * @return true if the namespace table manager is ready to serve, false
-   * otherwise
-   * @throws IOException
+   * @return true if the namespace table manager is ready to serve, false otherwise
    */
   @SuppressWarnings("deprecation")
-  public synchronized boolean isTableAvailableAndInitialized(
-      final boolean createNamespaceAync) throws IOException {
+  public synchronized boolean isTableAvailableAndInitialized()
+  throws IOException {
     // Did we already get a table? If so, still make sure it's available
     if (isTableNamespaceManagerInitialized()) {
       return true;
@@ -293,34 +330,10 @@ public class TableNamespaceManager {
         zkNamespaceManager.start();
 
         if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
-          if (createNamespaceAync) {
-            masterServices.getMasterProcedureExecutor().submitProcedure(
-              new CreateNamespaceProcedure(
-                masterServices.getMasterProcedureExecutor().getEnvironment(),
-                NamespaceDescriptor.DEFAULT_NAMESPACE));
-            initGoodSofar = false;
-          }
-          else {
-            masterServices.createNamespaceSync(
-              NamespaceDescriptor.DEFAULT_NAMESPACE,
-              HConstants.NO_NONCE,
-              HConstants.NO_NONCE);
-          }
+          blockingCreateNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
         }
         if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
-          if (createNamespaceAync) {
-            masterServices.getMasterProcedureExecutor().submitProcedure(
-              new CreateNamespaceProcedure(
-                masterServices.getMasterProcedureExecutor().getEnvironment(),
-                NamespaceDescriptor.SYSTEM_NAMESPACE));
-            initGoodSofar = false;
-          }
-          else {
-            masterServices.createNamespaceSync(
-              NamespaceDescriptor.SYSTEM_NAMESPACE,
-              HConstants.NO_NONCE,
-              HConstants.NO_NONCE);
-          }
+          blockingCreateNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
         }
 
         if (!initGoodSofar) {
@@ -410,4 +423,4 @@ public class TableNamespaceManager {
     }
     return maxRegions;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index 657bbfb..f934737 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -348,7 +348,7 @@ public class CreateNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getTableNamespaceManager();
+    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index 5a42614..2f99167 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -383,7 +383,7 @@ public class DeleteNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getTableNamespaceManager();
+    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 1e86254..baef112 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -374,7 +374,7 @@ public class DeleteTableProcedure
    */
   private static void cleanAnyRemainingRows(final MasterProcedureEnv env,
       final TableName tableName) throws IOException {
-    ClusterConnection connection = env.getMasterServices().getConnection();
+    Connection connection = env.getMasterServices().getConnection();
     Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName);
     try (Table metaTable =
         connection.getTable(TableName.META_TABLE_NAME)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
index 30de252..0f8c172 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
@@ -266,8 +266,9 @@ public class ModifyNamespaceProcedure
   }
 
   private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getTableNamespaceManager();
+    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }
+
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 5c9f6f4..bdcd89c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -730,7 +730,7 @@ implements ServerProcedureInterface {
     boolean metaAssigned = false;
     // Is hbase:meta location available yet?
     if (mtl.isLocationAvailable(zkw)) {
-      ClusterConnection connection = env.getMasterServices().getConnection();
+      ClusterConnection connection = env.getMasterServices().getClusterConnection();
       // Is hbase:meta location good yet?
       long timeout =
         env.getMasterConfiguration().getLong(KEY_SHORT_WAIT_ON_META, DEFAULT_SHORT_WAIT_ON_META);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
index f24f8c0..8035d32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
@@ -65,7 +65,7 @@ class NamespaceStateManager {
 
   /**
    * Gets an instance of NamespaceTableAndRegionInfo associated with namespace.
-   * @param The name of the namespace
+   * @param name The name of the namespace
    * @return An instance of NamespaceTableAndRegionInfo.
    */
   public NamespaceTableAndRegionInfo getState(String name) {
@@ -135,7 +135,7 @@ class NamespaceStateManager {
 
   private NamespaceDescriptor getNamespaceDescriptor(String namespaceAsString) {
     try {
-      return this.master.getNamespaceDescriptor(namespaceAsString);
+      return this.master.getClusterSchema().getNamespace(namespaceAsString);
     } catch (IOException e) {
       LOG.error("Error while fetching namespace descriptor for namespace : " + namespaceAsString);
       return null;
@@ -212,7 +212,7 @@ class NamespaceStateManager {
    * Initialize namespace state cache by scanning meta table.
    */
   private void initialize() throws IOException {
-    List<NamespaceDescriptor> namespaces = this.master.listNamespaceDescriptors();
+    List<NamespaceDescriptor> namespaces = this.master.getClusterSchema().getNamespaces();
     for (NamespaceDescriptor namespace : namespaces) {
       addNamespace(namespace.getName());
       List<TableName> tables = this.master.listTableNamesByNamespace(namespace.getName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 211fed5..00046ba 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
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
@@ -196,8 +197,7 @@ import sun.misc.SignalHandler;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HRegionServer extends HasThread implements
-    RegionServerServices, LastSequenceId {
+public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId {
 
   private static final Log LOG = LogFactory.getLog(HRegionServer.class);
 
@@ -1867,7 +1867,12 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public ClusterConnection getConnection() {
+  public Connection getConnection() {
+    return getClusterConnection();
+  }
+
+  @Override
+  public ClusterConnection getClusterConnection() {
     return this.clusterConnection;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 8d38b09..b86de12 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -187,5 +187,11 @@ public class ReplicationSyncUp extends Configured implements Tool {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index f132b2b..878c5bf 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -124,7 +124,7 @@
 </div>
 <%
 if ( fqtn != null ) {
-  table = (HTable) master.getConnection().getTable(fqtn);
+  table = (HTable) master.getConnection().getTable(TableName.valueOf(fqtn));
   if (table.getTableDescriptor().getRegionReplication() > 1) {
     tableHeader = "<h2>Table Regions</h2><table class=\"table table-striped\" style=\"table-layout: fixed; word-wrap: break-word;\"><tr><th style=\"width:22%\">Name</th><th>Region Server</th><th style=\"width:22%\">Start Key</th><th style=\"width:22%\">End Key</th><th>Locality</th><th>Requests</th><th>ReplicaID</th></tr>";
     withReplica = true;
@@ -199,7 +199,7 @@ if ( fqtn != null ) {
 </table>
 <%} else {
   Admin admin = master.getConnection().getAdmin();
-  RegionLocator r = master.getConnection().getRegionLocator(table.getName());
+  RegionLocator r = master.getClusterConnection().getRegionLocator(table.getName());
   try { %>
 <h2>Table Attributes</h2>
 <table class="table table-striped">

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index c126b19..a7fc75b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -306,4 +306,10 @@ public class MockRegionServerServices implements RegionServerServices {
   public double getCompactionPressure() {
     return 0;
   }
+
+  @Override
+  public ClusterConnection getClusterConnection() {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index c24d8a3..f9e2a16 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -358,7 +358,7 @@ public class TestNamespace {
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        admin.listTableDescriptorsByNamespace("non_existing_namespace");
+        admin.listTableDescriptorsByNamespace("non_existant_namespace");
         return null;
       }
     }, NamespaceNotFoundException.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
index e84d34c..618717b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
@@ -65,7 +65,7 @@ public class TestShortCircuitConnection {
     htd.addFamily(hcd);
     UTIL.createTable(htd, null);
     HRegionServer regionServer = UTIL.getRSForFirstRegionInTable(tn);
-    ClusterConnection connection = regionServer.getConnection();
+    ClusterConnection connection = regionServer.getClusterConnection();
     HTableInterface tableIf = connection.getTable(tn);
     assertTrue(tableIf instanceof HTable);
     HTable table = (HTable) tableIf;

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index ef4a579..638811a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -1586,7 +1586,13 @@ public class TestMasterObserver {
     cp.enableBypass(true);
     cp.resetStates();
 
-    admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build());
+    boolean expected = false;
+    try {
+      admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build());
+    } catch (BypassCoprocessorException ce) {
+      expected = true;
+    }
+    assertTrue(expected);
     assertTrue("Test namespace should not have been modified",
         cp.preModifyNamespaceCalledOnly());
 
@@ -1594,7 +1600,13 @@ public class TestMasterObserver {
     assertTrue("Test namespace descriptor should have been called",
         cp.wasGetNamespaceDescriptorCalled());
 
-    admin.deleteNamespace(testNamespace);
+    expected = false;
+    try {
+      admin.deleteNamespace(testNamespace);
+    } catch (BypassCoprocessorException ce) {
+      expected = true;
+    }
+    assertTrue(expected);
     assertTrue("Test namespace should not have been deleted", cp.preDeleteNamespaceCalledOnly());
 
     assertNotNull(admin.getNamespaceDescriptor(testNamespace));
@@ -1614,7 +1626,13 @@ public class TestMasterObserver {
     cp.enableBypass(true);
     cp.resetStates();
 
-    admin.createNamespace(NamespaceDescriptor.create(testNamespace).build());
+    expected = false;
+    try {
+      admin.createNamespace(NamespaceDescriptor.create(testNamespace).build());
+    } catch (BypassCoprocessorException ce) {
+      expected = true;
+    }
+    assertTrue(expected);
     assertTrue("Test namespace should not be created", cp.preCreateNamespaceCalledOnly());
 
     // turn on bypass, run the test

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index eb8f803..234ad20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -651,4 +651,10 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   public double getCompactionPressure() {
     return 0;
   }
+
+  @Override
+  public ClusterConnection getClusterConnection() {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index e3283e9..e10ab2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -326,5 +326,11 @@ public class TestActiveMasterManager {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 8e35bbf..e26bd82 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaMockingUtil;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -143,11 +142,10 @@ public class TestCatalogJanitor {
             ServerName.valueOf("example.org,12345,6789"),
           HRegionInfo.FIRST_META_REGIONINFO);
       // Set hbase.rootdir into test dir.
-      FileSystem fs = FileSystem.get(this.c);
+      FileSystem.get(this.c);
       Path rootdir = FSUtils.getRootDir(this.c);
       FSUtils.setRootDir(this.c, rootdir);
-      AdminProtos.AdminService.BlockingInterface hri =
-        Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
+      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
     }
 
     @Override
@@ -208,6 +206,12 @@ public class TestCatalogJanitor {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   /**
@@ -402,48 +406,6 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public void createNamespace(
-        final NamespaceDescriptor descriptor,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public void createNamespaceSync(
-        final NamespaceDescriptor descriptor,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public void modifyNamespace(
-        final NamespaceDescriptor descriptor,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public void deleteNamespace(
-        final String name,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
-      return null;  //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
-      return null;  //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
     public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
         throws IOException {
       return false;  //To change body of implemented methods use File | Settings | File Templates.
@@ -536,32 +498,35 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public TableNamespaceManager getTableNamespaceManager() {
-      return null;
-    }
-
-    @Override
     public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
         boolean forcible) throws IOException {
     }
 
     @Override
     public boolean isInitialized() {
-      // Auto-generated method stub
       return false;
     }
 
     @Override
     public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
-      // Auto-generated method stub
       return 0;
     }
 
     @Override
     public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
-      // Auto-generated method stub
       return 0;
     }
+
+    @Override
+    public ClusterSchema getClusterSchema() {
+      return null;
+    }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
index a19d5d8..142437c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
@@ -100,6 +100,12 @@ public class TestClockSkewDetection {
       public ChoreService getChoreService() {
         return null;
       }
+
+      @Override
+      public ClusterConnection getClusterConnection() {
+        // TODO Auto-generated method stub
+        return null;
+      }
     }, null, false);
 
     LOG.debug("regionServerStartup 1");

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 972834a..398a898 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -32,6 +32,7 @@ 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.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
@@ -62,8 +63,10 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 import org.mockito.Mockito;
 
 import com.google.protobuf.ServiceException;
@@ -80,6 +83,8 @@ import com.google.protobuf.ServiceException;
 public class TestMasterNoCluster {
   private static final Log LOG = LogFactory.getLog(TestMasterNoCluster.class);
   private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+      withTimeout(this.getClass()).withLookingForStuckThread(true).build();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -122,7 +127,7 @@ public class TestMasterNoCluster {
    * @throws KeeperException
    * @throws InterruptedException
    */
-  @Test (timeout=30000)
+  @Test
   public void testStopDuringStart()
   throws IOException, KeeperException, InterruptedException {
     CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
@@ -141,7 +146,7 @@ public class TestMasterNoCluster {
    * @throws KeeperException
    * @throws InterruptedException
    */
-  @Test (timeout=30000)
+  @Test
   public void testFailover()
   throws IOException, KeeperException, InterruptedException, ServiceException {
     final long now = System.currentTimeMillis();
@@ -193,6 +198,9 @@ public class TestMasterNoCluster {
       }
 
       @Override
+      void initClusterSchemaService() throws IOException, InterruptedException {}
+
+      @Override
       ServerManager createServerManager(Server master, MasterServices services)
       throws IOException {
         ServerManager sm = super.createServerManager(master, services);
@@ -218,10 +226,6 @@ public class TestMasterNoCluster {
           return null;
         }
       }
-
-      @Override
-      void initNamespace() {
-      }
     };
     master.start();
 
@@ -266,6 +270,9 @@ public class TestMasterNoCluster {
       { }
 
       @Override
+      void initClusterSchemaService() throws IOException, InterruptedException {}
+
+      @Override
       void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException, CoordinatedStateException {
         super.initializeZKBasedSystemTrackers();
@@ -294,10 +301,6 @@ public class TestMasterNoCluster {
           return null;
         }
       }
-
-      @Override
-      void initNamespace() {
-      }
     };
     master.start();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
index 65c8649..c7707b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
@@ -168,6 +168,12 @@ public class TestSplitLogManager {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   static Stoppable stopper = new Stoppable() {


[27/50] [abbrv] hbase git commit: HBASE-15066 Small improvements to Canary tool

Posted by jm...@apache.org.
HBASE-15066 Small improvements to Canary tool


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

Branch: refs/heads/trunk
Commit: 893a54c3a40e08b6f50d753fca01688f491893d0
Parents: 073e00c
Author: Enis Soztutar <en...@apache.org>
Authored: Wed Jan 6 02:47:19 2016 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Wed Jan 6 02:47:19 2016 -0800

----------------------------------------------------------------------
 bin/hbase                                       |  4 ++
 .../org/apache/hadoop/hbase/tool/Canary.java    | 72 ++++++++++++++++----
 src/main/asciidoc/_chapters/ops_mgt.adoc        | 20 +++---
 3 files changed, 72 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/893a54c3/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 5064451..7742b5b 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -98,6 +98,7 @@ if [ $# = 0 ]; then
   echo "  mapredcp        Dump CLASSPATH entries required by mapreduce"
   echo "  pe              Run PerformanceEvaluation"
   echo "  ltt             Run LoadTestTool"
+  echo "  canary          Run the Canary tool"
   echo "  version         Print the version"
   echo "  CLASSNAME       Run the class named CLASSNAME"
   exit 1
@@ -368,6 +369,9 @@ elif [ "$COMMAND" = "pe" ] ; then
 elif [ "$COMMAND" = "ltt" ] ; then
   CLASS='org.apache.hadoop.hbase.util.LoadTestTool'
   HBASE_OPTS="$HBASE_OPTS $HBASE_LTT_OPTS"
+elif [ "$COMMAND" = "canary" ] ; then
+  CLASS='org.apache.hadoop.hbase.tool.Canary'
+  HBASE_OPTS="$HBASE_OPTS $HBASE_CANARY_OPTS"
 elif [ "$COMMAND" = "version" ] ; then
   CLASS='org.apache.hadoop.hbase.util.VersionInfo'
 else

http://git-wip-us.apache.org/repos/asf/hbase/blob/893a54c3/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index a1c8c61..151be42 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -195,6 +195,10 @@ public final class Canary implements Tool {
       Table table = null;
       HTableDescriptor tableDesc = null;
       try {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("reading table descriptor for table %s",
+            region.getTable()));
+        }
         table = connection.getTable(region.getTable());
         tableDesc = table.getTableDescriptor();
       } catch (IOException e) {
@@ -232,20 +236,24 @@ public final class Canary implements Tool {
           scan.setFilter(new FirstKeyOnlyFilter());
           scan.addFamily(column.getName());
           scan.setMaxResultSize(1L);
+          scan.setSmall(true);
         }
 
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("reading from table %s region %s column family %s and key %s",
+            tableDesc.getTableName(), region.getRegionNameAsString(), column.getNameAsString(),
+            Bytes.toStringBinary(startKey)));
+        }
         try {
+          stopWatch.start();
           if (startKey.length > 0) {
-            stopWatch.start();
             table.get(get);
-            stopWatch.stop();
-            sink.publishReadTiming(region, column, stopWatch.getTime());
           } else {
-            stopWatch.start();
             rs = table.getScanner(scan);
-            stopWatch.stop();
-            sink.publishReadTiming(region, column, stopWatch.getTime());
+            rs.next();
           }
+          stopWatch.stop();
+          sink.publishReadTiming(region, column, stopWatch.getTime());
         } catch (Exception e) {
           sink.publishReadFailure(region, column, e);
         } finally {
@@ -286,6 +294,12 @@ public final class Canary implements Tool {
           byte[] value = new byte[writeValueSize];
           Bytes.random(value);
           put.addColumn(column.getName(), HConstants.EMPTY_BYTE_ARRAY, value);
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("writing to table %s region %s column family %s and key %s",
+              tableDesc.getTableName(), region.getRegionNameAsString(), column.getNameAsString(),
+              Bytes.toStringBinary(rowToCheck)));
+          }
           try {
             long startTime = System.currentTimeMillis();
             table.put(put);
@@ -337,6 +351,11 @@ public final class Canary implements Tool {
         table = connection.getTable(tableName);
         startKey = region.getStartKey();
         // Can't do a get on empty start row so do a Scan of first element if any instead.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("reading from region server %s table %s region %s and key %s",
+            serverName, region.getTable(), region.getRegionNameAsString(),
+            Bytes.toStringBinary(startKey)));
+        }
         if (startKey.length > 0) {
           get = new Get(startKey);
           get.setCacheBlocks(false);
@@ -350,8 +369,10 @@ public final class Canary implements Tool {
           scan.setFilter(new FirstKeyOnlyFilter());
           scan.setCaching(1);
           scan.setMaxResultSize(1L);
+          scan.setSmall(true);
           stopWatch.start();
           ResultScanner s = table.getScanner(scan);
+          s.next();
           s.close();
           stopWatch.stop();
         }
@@ -547,8 +568,6 @@ public final class Canary implements Tool {
     long startTime = 0;
     long currentTimeLength = 0;
     // Get a connection to use in below.
-    // try-with-resources jdk7 construct. See
-    // http://docs.oracle.com/javase/tutorial/essential/exceptions/tryResourceClose.html
     try (Connection connection = ConnectionFactory.createConnection(this.conf)) {
       do {
         // Do monitor !!
@@ -613,8 +632,8 @@ public final class Canary implements Tool {
     System.err.println("      only works in regionserver mode.");
     System.err.println("   -daemon        Continuous check at defined intervals.");
     System.err.println("   -interval <N>  Interval between checks (sec)");
-    System.err.println("   -e             Use region/regionserver as regular expression");
-    System.err.println("      which means the region/regionserver is regular expression pattern");
+    System.err.println("   -e             Use table/regionserver as regular expression");
+    System.err.println("      which means the table/regionserver is regular expression pattern");
     System.err.println("   -f <B>         stop whole program if first error occurs," +
         " default is true");
     System.err.println("   -t <N>         timeout for a check, default is 600000 (milisecs)");
@@ -691,6 +710,7 @@ public final class Canary implements Tool {
       this.executor = executor;
     }
 
+    @Override
     public abstract void run();
 
     protected boolean initAdmin() {
@@ -793,11 +813,17 @@ public final class Canary implements Tool {
         HTableDescriptor[] tds = null;
         Set<String> tmpTables = new TreeSet<String>();
         try {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("reading list of tables"));
+          }
+          tds = this.admin.listTables(pattern);
+          if (tds == null) {
+            tds = new HTableDescriptor[0];
+          }
           for (String monitorTarget : monitorTargets) {
             pattern = Pattern.compile(monitorTarget);
-            tds = this.admin.listTables(pattern);
-            if (tds != null) {
-              for (HTableDescriptor td : tds) {
+            for (HTableDescriptor td : tds) {
+              if (pattern.matcher(td.getNameAsString()).matches()) {
                 tmpTables.add(td.getNameAsString());
               }
             }
@@ -826,6 +852,9 @@ public final class Canary implements Tool {
      * canary entry point to monitor all the tables.
      */
     private List<Future<Void>> sniff(TaskType taskType) throws Exception {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("reading list of tables"));
+      }
       List<Future<Void>> taskFutures = new LinkedList<Future<Void>>();
       for (HTableDescriptor table : admin.listTables()) {
         if (admin.isTableEnabled(table.getTableName())
@@ -873,7 +902,7 @@ public final class Canary implements Tool {
       int numberOfRegions = (int)(numberOfServers * regionsLowerLimit);
       LOG.info("Number of live regionservers: " + numberOfServers + ", "
           + "pre-splitting the canary table into " + numberOfRegions + " regions "
-          + "(current  lower limi of regions per server is " + regionsLowerLimit
+          + "(current lower limit of regions per server is " + regionsLowerLimit
           + " and you can change it by config: "
           + HConstants.HBASE_CANARY_WRITE_PERSERVER_REGIONS_LOWERLIMIT_KEY + " )");
       HTableDescriptor desc = new HTableDescriptor(writeTableName);
@@ -916,6 +945,10 @@ public final class Canary implements Tool {
    */
   private static List<Future<Void>> sniff(final Admin admin, final Sink sink, String tableName,
       ExecutorService executor, TaskType taskType) throws Exception {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("checking table is enabled and getting table descriptor for table %s",
+        tableName));
+    }
     if (admin.isTableEnabled(TableName.valueOf(tableName))) {
       return Canary.sniff(admin, sink, admin.getTableDescriptor(TableName.valueOf(tableName)),
         executor, taskType);
@@ -930,6 +963,11 @@ public final class Canary implements Tool {
    */
   private static List<Future<Void>> sniff(final Admin admin, final Sink sink,
       HTableDescriptor tableDesc, ExecutorService executor, TaskType taskType) throws Exception {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("reading list of regions for table %s", tableDesc.getTableName()));
+    }
+
     Table table = null;
     try {
       table = admin.getConnection().getTable(tableDesc.getTableName());
@@ -975,6 +1013,9 @@ public final class Canary implements Tool {
       List<String> foundTableNames = new ArrayList<String>();
       TableName[] tableNames = null;
 
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("reading list of tables"));
+      }
       try {
         tableNames = this.admin.listTableNames();
       } catch (IOException e) {
@@ -1060,6 +1101,9 @@ public final class Canary implements Tool {
       Table table = null;
       RegionLocator regionLocator = null;
       try {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("reading list of tables and locations"));
+        }
         HTableDescriptor[] tableDescs = this.admin.listTables();
         List<HRegionInfo> regions = null;
         for (HTableDescriptor tableDesc : tableDescs) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/893a54c3/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 e8d44eb..13835c0 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -79,7 +79,7 @@ There is a Canary class can help users to canary-test the HBase cluster status,
 To see the usage, use the `--help` parameter.
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -help
+$ ${HBASE_HOME}/bin/hbase canary -help
 
 Usage: bin/hbase org.apache.hadoop.hbase.tool.Canary [opts] [table1 [table2]...] | [regionserver1 [regionserver2]..]
  where [opts] are:
@@ -126,7 +126,7 @@ Following are some examples based on the previous given case.
 ==== Canary test for every column family (store) of every region of every table
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary
+$ ${HBASE_HOME}/bin/hbase canary
 
 3/12/09 03:26:32 INFO tool.Canary: read from region test-01,,1386230156732.0e3c7d77ffb6361ea1b996ac1042ca9a. column family cf1 in 2ms
 13/12/09 03:26:32 INFO tool.Canary: read from region test-01,,1386230156732.0e3c7d77ffb6361ea1b996ac1042ca9a. column family cf2 in 2ms
@@ -147,7 +147,7 @@ This is a default behavior of the this tool does.
 You can also test one or more specific tables.
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary test-01 test-02
+$ ${HBASE_HOME}/bin/hbase canary test-01 test-02
 ----
 
 ==== Canary test with RegionServer granularity
@@ -155,7 +155,7 @@ $ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary test-01 test-02
 This will pick one small piece of data from each RegionServer, and can also put your RegionServer name as input options for canary-test specific RegionServer.
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -regionserver
+$ ${HBASE_HOME}/bin/hbase canary -regionserver
 
 13/12/09 06:05:17 INFO tool.Canary: Read from table:test-01 on region server:rs2 in 72ms
 13/12/09 06:05:17 INFO tool.Canary: Read from table:test-02 on region server:rs3 in 34ms
@@ -167,7 +167,7 @@ $ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -regionserver
 This will test both table test-01 and test-02.
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -e test-0[1-2]
+$ ${HBASE_HOME}/bin/hbase canary -e test-0[1-2]
 ----
 
 ==== Run canary test as daemon mode
@@ -176,13 +176,13 @@ Run repeatedly with interval defined in option `-interval` whose default value i
 This daemon will stop itself and return non-zero error code if any error occurs, due to the default value of option -f is true.
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -daemon
+$ ${HBASE_HOME}/bin/hbase canary -daemon
 ----
 
 Run repeatedly with internal 5 seconds and will not stop itself even if errors occur in the test.
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -daemon -interval 50000 -f false
+$ ${HBASE_HOME}/bin/hbase canary -daemon -interval 50000 -f false
 ----
 
 ==== Force timeout if canary test stuck
@@ -192,7 +192,7 @@ Because of this we provide a timeout option to kill the canary test and return a
 This run sets the timeout value to 60 seconds, the default value is 600 seconds.
 
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -t 600000
+$ ${HBASE_HOME}/bin/hbase canary -t 600000
 ----
 
 ==== Enable write sniffing in canary
@@ -203,12 +203,12 @@ When the write sniffing is enabled, the canary tool will create an hbase table a
 regions of the table distributed on all region servers. In each sniffing period, the canary will
 try to put data to these regions to check the write availability of each region server.
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -writeSniffing
+$ ${HBASE_HOME}/bin/hbase canary -writeSniffing
 ----
 
 The default write table is `hbase:canary` and can be specified by the option `-writeTable`.
 ----
-$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -writeSniffing -writeTable ns:canary
+$ ${HBASE_HOME}/bin/hbase canary -writeSniffing -writeTable ns:canary
 ----
 
 The default value size of each put is 10 bytes and you can set it by the config key:


[48/50] [abbrv] hbase git commit: HBASE-14159 Add dependency on hadoop-mapreduce-client-jobclient from hbase-spark.

Posted by jm...@apache.org.
HBASE-14159 Add dependency on hadoop-mapreduce-client-jobclient from hbase-spark.

Adds a test-scope dependency to get rid of following error:

[WARNING] warning: Class org.apache.hadoop.mapred.MiniMRCluster not found - continuing with a stub.
[WARNING] one warning found

Signed-off-by: Sean Busbey <bu...@cloudera.com>


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

Branch: refs/heads/trunk
Commit: 5e89ebcc2f37366fd1efa7f2317a60552cf9e2b8
Parents: 840f5ea
Author: Apekshit <ap...@gmail.com>
Authored: Mon Jan 11 12:04:53 2016 -0800
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Tue Jan 12 07:48:42 2016 -0600

----------------------------------------------------------------------
 hbase-spark/pom.xml | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5e89ebcc/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 2efbca0..8f71a89 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -520,6 +520,12 @@
             <groupId>commons-logging</groupId>
             <artifactId>commons-logging</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
 
@@ -600,10 +606,10 @@
         </plugins>
     </build>
 
-  <profiles>
+    <profiles>
     <!-- Skip the tests in this module -->
-      <profile>
-        <id>skipSparkTests</id>
+        <profile>
+            <id>skipSparkTests</id>
             <activation>
                 <property>
                     <name>skipSparkTests</name>


[33/50] [abbrv] hbase git commit: HBASE-15070 DistributedHBaseCluster#restoreRegionServers() starts new RS process on master server (Samir Ahmic)

Posted by jm...@apache.org.
HBASE-15070 DistributedHBaseCluster#restoreRegionServers() starts new RS process on master server (Samir Ahmic)


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

Branch: refs/heads/trunk
Commit: 3d3677932a4ec98c12121c879ac5e2ea71925ea5
Parents: 5266b07
Author: tedyu <yu...@gmail.com>
Authored: Thu Jan 7 09:14:00 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Jan 7 09:14:00 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/3d367793/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 62f5c66..6f3baa0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -433,6 +433,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
     Set<ServerName> toKill = new TreeSet<ServerName>(new ServerNameIgnoreStartCodeComparator());
     toStart.addAll(initial.getServers());
     toKill.addAll(current.getServers());
+    
+    ServerName master = initial.getMaster();
 
     for (ServerName server : current.getServers()) {
       toStart.remove(server);
@@ -447,7 +449,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
       try {
         if (!clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER,
                 sn.getHostname(),
-                sn.getPort())) {
+                sn.getPort())
+                && master.getPort() != sn.getPort()) {
           LOG.info("Restoring cluster - starting initial region server: " + sn.getHostAndPort());
           startRegionServer(sn.getHostname(), sn.getPort());
         }
@@ -460,7 +463,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
       try {
         if (clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER,
                 sn.getHostname(),
-                sn.getPort())) {
+                sn.getPort())
+                && master.getPort() != sn.getPort()){
           LOG.info("Restoring cluster - stopping initial region server: " + sn.getHostAndPort());
           stopRegionServer(sn);
         }


[35/50] [abbrv] hbase git commit: HBASE-15079 TestMultiParallel.validateLoadedData AssertionError: null

Posted by jm...@apache.org.
HBASE-15079 TestMultiParallel.validateLoadedData AssertionError: null


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

Branch: refs/heads/trunk
Commit: f3ee6df0f2d0955c2b334a9131eb3994c00af0c4
Parents: 5bde960
Author: stack <st...@apache.org>
Authored: Thu Jan 7 15:35:59 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Jan 7 15:36:49 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/TestMultiParallel.java  | 39 ++++++++++++++++++--
 1 file changed, 35 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f3ee6df0/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index b2c5c11..af3a54e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -683,13 +683,44 @@ public class TestMultiParallel {
   private void validateLoadedData(Table table) throws IOException {
     // get the data back and validate that it is correct
     LOG.info("Validating data on " + table);
+    List<Get> gets = new ArrayList<Get>();
     for (byte[] k : KEYS) {
       Get get = new Get(k);
       get.addColumn(BYTES_FAMILY, QUALIFIER);
-      Result r = table.get(get);
-      Assert.assertTrue(r.containsColumn(BYTES_FAMILY, QUALIFIER));
-      Assert.assertEquals(0, Bytes.compareTo(VALUE, r
-          .getValue(BYTES_FAMILY, QUALIFIER)));
+      gets.add(get);
+    }
+    int retryNum = 10;
+    Result[] results = null;
+    do  {
+      results = table.get(gets);
+      boolean finished = true;
+      for (Result result : results) {
+        if (result.isEmpty()) {
+          finished = false;
+          break;
+        }
+      }
+      if (finished) {
+        break;
+      }
+      try {
+        Thread.sleep(10);
+      } catch (InterruptedException e) {
+      }
+      retryNum--;
+    } while (retryNum > 0);
+
+    if (retryNum == 0) {
+      fail("Timeout for validate data");
+    } else {
+      if (results != null) {
+        for (Result r : results) {
+          Assert.assertTrue(r.containsColumn(BYTES_FAMILY, QUALIFIER));
+          Assert.assertEquals(0, Bytes.compareTo(VALUE, r
+            .getValue(BYTES_FAMILY, QUALIFIER)));
+        }
+        LOG.info("Validating data on " + table + " successfully!");
+      }
     }
   }
 


[47/50] [abbrv] hbase git commit: HBASE-15085 IllegalStateException was thrown when scanning on bulkloaded HFiles (Victor Xu)

Posted by jm...@apache.org.
HBASE-15085 IllegalStateException was thrown when scanning on bulkloaded
HFiles (Victor Xu)


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

Branch: refs/heads/trunk
Commit: 840f5ea686e50c4b8d81b3fe1df2f3a7bd807636
Parents: 83c506d
Author: ramkrishna <ra...@gmail.com>
Authored: Tue Jan 12 14:36:48 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Tue Jan 12 14:36:48 2016 +0530

----------------------------------------------------------------------
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |  6 +++
 .../mapreduce/TestLoadIncrementalHFiles.java    | 46 ++++++++++++++++++++
 .../apache/hadoop/hbase/util/HFileTestUtil.java | 25 +++++++++--
 3 files changed, 73 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/840f5ea6/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 172dfe3..5d75d56 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -929,6 +930,11 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   }
 
   private static boolean shouldCopyHFileMetaKey(byte[] key) {
+    // skip encoding to keep hfile meta consistent with data block info, see HBASE-15085
+    if (Bytes.equals(key, HFileDataBlockEncoder.DATA_BLOCK_ENCODING)) {
+      return false;
+    }
+
     return !HFile.isReservedFileInfoKey(key);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/840f5ea6/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
index 8c64ff8..6dc8566 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@@ -482,6 +483,51 @@ public class TestLoadIncrementalHFiles {
     assertEquals(1000, rowCount);
   }
 
+  @Test
+  public void testSplitStoreFileWithNoneToNone() throws IOException {
+    testSplitStoreFileWithDifferentEncoding(DataBlockEncoding.NONE, DataBlockEncoding.NONE);
+  }
+
+  @Test
+  public void testSplitStoreFileWithEncodedToEncoded() throws IOException {
+    testSplitStoreFileWithDifferentEncoding(DataBlockEncoding.DIFF, DataBlockEncoding.DIFF);
+  }
+
+  @Test
+  public void testSplitStoreFileWithEncodedToNone() throws IOException {
+    testSplitStoreFileWithDifferentEncoding(DataBlockEncoding.DIFF, DataBlockEncoding.NONE);
+  }
+
+  @Test
+  public void testSplitStoreFileWithNoneToEncoded() throws IOException {
+    testSplitStoreFileWithDifferentEncoding(DataBlockEncoding.NONE, DataBlockEncoding.DIFF);
+  }
+
+  private void testSplitStoreFileWithDifferentEncoding(DataBlockEncoding bulkloadEncoding,
+      DataBlockEncoding cfEncoding) throws IOException {
+    Path dir = util.getDataTestDirOnTestFS("testSplitHFileWithDifferentEncoding");
+    FileSystem fs = util.getTestFileSystem();
+    Path testIn = new Path(dir, "testhfile");
+    HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY);
+    familyDesc.setDataBlockEncoding(cfEncoding);
+    HFileTestUtil.createHFileWithDataBlockEncoding(
+        util.getConfiguration(), fs, testIn, bulkloadEncoding,
+        FAMILY, QUALIFIER, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), 1000);
+
+    Path bottomOut = new Path(dir, "bottom.out");
+    Path topOut = new Path(dir, "top.out");
+
+    LoadIncrementalHFiles.splitStoreFile(
+        util.getConfiguration(), testIn,
+        familyDesc, Bytes.toBytes("ggg"),
+        bottomOut,
+        topOut);
+
+    int rowCount = verifyHFile(bottomOut);
+    rowCount += verifyHFile(topOut);
+    assertEquals(1000, rowCount);
+  }
+
   private int verifyHFile(Path p) throws IOException {
     Configuration conf = util.getConfiguration();
     HFile.Reader reader = HFile.createReader(

http://git-wip-us.apache.org/repos/asf/hbase/blob/840f5ea6/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
index 964d6ed..fbd79c3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
@@ -32,6 +32,7 @@ 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.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -59,7 +60,21 @@ public class HFileTestUtil {
       FileSystem fs, Path path,
       byte[] family, byte[] qualifier,
       byte[] startKey, byte[] endKey, int numRows) throws IOException {
-    createHFile(configuration, fs, path, family, qualifier, startKey, endKey,
+      createHFile(configuration, fs, path, DataBlockEncoding.NONE, family, qualifier,
+        startKey, endKey, numRows, false);
+  }
+
+  /**
+   * Create an HFile with the given number of rows between a given
+   * start key and end key @ family:qualifier.  The value will be the key value.
+   * This file will use certain data block encoding algorithm.
+   */
+  public static void createHFileWithDataBlockEncoding(
+      Configuration configuration,
+      FileSystem fs, Path path, DataBlockEncoding encoding,
+      byte[] family, byte[] qualifier,
+      byte[] startKey, byte[] endKey, int numRows) throws IOException {
+      createHFile(configuration, fs, path, encoding, family, qualifier, startKey, endKey,
         numRows, false);
   }
 
@@ -73,7 +88,8 @@ public class HFileTestUtil {
       FileSystem fs, Path path,
       byte[] family, byte[] qualifier,
       byte[] startKey, byte[] endKey, int numRows) throws IOException {
-    createHFile(configuration, fs, path, family, qualifier, startKey, endKey, numRows, true);
+      createHFile(configuration, fs, path, DataBlockEncoding.NONE, family, qualifier,
+        startKey, endKey, numRows, true);
   }
 
   /**
@@ -84,11 +100,12 @@ public class HFileTestUtil {
    */
   public static void createHFile(
       Configuration configuration,
-      FileSystem fs, Path path,
+      FileSystem fs, Path path, DataBlockEncoding encoding,
       byte[] family, byte[] qualifier,
       byte[] startKey, byte[] endKey, int numRows, boolean withTag) throws IOException {
     HFileContext meta = new HFileContextBuilder()
         .withIncludesTags(withTag)
+        .withDataBlockEncoding(encoding)
         .build();
     HFile.Writer writer = HFile.getWriterFactory(configuration, new CacheConfig(configuration))
         .withPath(fs, path)
@@ -141,4 +158,4 @@ public class HFileTestUtil {
       }
     }
   }
-}
\ No newline at end of file
+}


[32/50] [abbrv] hbase git commit: HBASE-15068 Add metrics for region normalization plans

Posted by jm...@apache.org.
HBASE-15068 Add metrics for region normalization plans


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

Branch: refs/heads/trunk
Commit: 5266b0770843c57e977385d7ea1d0ea40273668a
Parents: d65978f
Author: tedyu <yu...@gmail.com>
Authored: Thu Jan 7 03:13:16 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Jan 7 03:13:16 2016 -0800

----------------------------------------------------------------------
 .../hbase/master/MetricsMasterSource.java       |  5 ++++-
 .../hbase/master/MetricsMasterWrapper.java      |  9 ++++++++
 .../hbase/master/MetricsMasterSourceImpl.java   |  4 ++++
 .../org/apache/hadoop/hbase/master/HMaster.java | 22 ++++++++++++++++++++
 .../hbase/master/MetricsMasterWrapperImpl.java  | 10 +++++++++
 .../hbase/master/TestMasterMetricsWrapper.java  |  2 ++
 6 files changed, 51 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5266b077/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
index ab621cc..290b8f5 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
@@ -57,6 +57,8 @@ public interface MetricsMasterSource extends BaseSource {
   String SERVER_NAME_NAME = "serverName";
   String CLUSTER_ID_NAME = "clusterId";
   String IS_ACTIVE_MASTER_NAME = "isActiveMaster";
+  String SPLIT_PLAN_COUNT_NAME = "splitPlanCount";
+  String MERGE_PLAN_COUNT_NAME = "mergePlanCount";
 
   String CLUSTER_REQUESTS_NAME = "clusterRequests";
   String MASTER_ACTIVE_TIME_DESC = "Master Active Time";
@@ -70,7 +72,8 @@ public interface MetricsMasterSource extends BaseSource {
   String SERVER_NAME_DESC = "Server Name";
   String CLUSTER_ID_DESC = "Cluster Id";
   String IS_ACTIVE_MASTER_DESC = "Is Active Master";
-
+  String SPLIT_PLAN_COUNT_DESC = "Number of Region Split Plans executed";
+  String MERGE_PLAN_COUNT_DESC = "Number of Region Merge Plans executed";
 
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/5266b077/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
index 678db69..5e67f83 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
@@ -112,4 +112,13 @@ public interface MetricsMasterWrapper {
    */
   long getNumWALFiles();
 
+  /**
+   * Get the number of region split plans executed.
+   */
+  long getSplitPlanCount();
+
+  /**
+   * Get the number of region merge plans executed.
+   */
+  long getMergePlanCount();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5266b077/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
index c5ce5e4..b0ba66e 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
@@ -74,6 +74,10 @@ public class MetricsMasterSourceImpl
     // masterWrapper can be null because this function is called inside of init.
     if (masterWrapper != null) {
       metricsRecordBuilder
+          .addGauge(Interns.info(MERGE_PLAN_COUNT_NAME, MERGE_PLAN_COUNT_DESC),
+              masterWrapper.getMergePlanCount())
+          .addGauge(Interns.info(SPLIT_PLAN_COUNT_NAME, SPLIT_PLAN_COUNT_DESC),
+              masterWrapper.getSplitPlanCount())
           .addGauge(Interns.info(MASTER_ACTIVE_TIME_NAME,
               MASTER_ACTIVE_TIME_DESC), masterWrapper.getActiveTime())
           .addGauge(Interns.info(MASTER_START_TIME_NAME,

http://git-wip-us.apache.org/repos/asf/hbase/blob/5266b077/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 4a9b792..8ff7ab1 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
@@ -333,6 +333,9 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   // handle table states
   private TableStateManager tableStateManager;
+  
+  private long splitPlanCount;
+  private long mergePlanCount;
 
   /** flag used in test cases in order to simulate RS failures during master initialization */
   private volatile boolean initializationBeforeMetaAssignment = false;
@@ -1340,6 +1343,11 @@ public class HMaster extends HRegionServer implements MasterServices {
         }
         NormalizationPlan plan = this.normalizer.computePlanForTable(table, types);
         plan.execute(clusterConnection.getAdmin());
+        if (plan.getType() == PlanType.SPLIT) {
+          splitPlanCount++;
+        } else if (plan.getType() == PlanType.MERGE) {
+          mergePlanCount++;
+        }
       }
     }
     // If Region did not generate any plans, it means the cluster is already balanced.
@@ -2335,6 +2343,20 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     return regionStates.getAverageLoad();
   }
+  
+  /*
+   * @return the count of region split plans executed
+   */
+  public long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /*
+   * @return the count of region merge plans executed
+   */
+  public long getMergePlanCount() {
+    return mergePlanCount;
+  }
 
   @Override
   public boolean registerService(Service instance) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5266b077/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
index a935a37..4cff28b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
@@ -40,6 +40,16 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
   }
 
   @Override
+  public long getSplitPlanCount() {
+    return master.getSplitPlanCount();
+  }
+
+  @Override
+  public long getMergePlanCount() {
+    return master.getMergePlanCount();
+  }
+
+  @Override
   public String getClusterId() {
     return master.getClusterId();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5266b077/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
index 2df4ac9..02f3721 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
@@ -50,6 +50,8 @@ public class TestMasterMetricsWrapper {
   public void testInfo() {
     HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
     MetricsMasterWrapperImpl info = new MetricsMasterWrapperImpl(master);
+    assertEquals(master.getSplitPlanCount(), info.getSplitPlanCount(), 0);
+    assertEquals(master.getMergePlanCount(), info.getMergePlanCount(), 0);
     assertEquals(master.getAverageLoad(), info.getAverageLoad(), 0);
     assertEquals(master.getClusterId(), info.getClusterId());
     assertEquals(master.getMasterActiveTime(), info.getActiveTime());


[24/50] [abbrv] hbase git commit: HBASE-14468 addendum.

Posted by jm...@apache.org.
HBASE-14468 addendum.


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

Branch: refs/heads/trunk
Commit: e8fbc9b43a3742358e0bdfe441ff4ca9d14e127b
Parents: 72d32cc
Author: Lars Hofhansl <la...@apache.org>
Authored: Tue Jan 5 15:54:34 2016 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Tue Jan 5 15:54:34 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java    | 17 ++---------------
 1 file changed, 2 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e8fbc9b4/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 4472b65..2e42acb 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
@@ -1607,21 +1607,8 @@ public class HMaster extends HRegionServer implements MasterServices {
             ExploringCompactionPolicy.class.getName());
     }
 
-    long majorCompactionPeriod = Long.MAX_VALUE;
-    String sv = htd.getConfigurationValue(HConstants.MAJOR_COMPACTION_PERIOD);
-    if (sv != null) {
-      majorCompactionPeriod = Long.parseLong(sv);
-    } else {
-      majorCompactionPeriod =
-          conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, majorCompactionPeriod);
-    }
-    String splitPolicyClassName = htd.getRegionSplitPolicyClassName();
-    if (splitPolicyClassName == null) {
-      splitPolicyClassName = conf.get(HConstants.HBASE_REGION_SPLIT_POLICY_KEY);
-    }
-
     int blockingFileCount = HStore.DEFAULT_BLOCKING_STOREFILE_COUNT;
-    sv = htd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY);
+    String sv = htd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY);
     if (sv != null) {
       blockingFileCount = Integer.parseInt(sv);
     } else {
@@ -1634,7 +1621,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       if (compactionPolicy == null) {
         compactionPolicy = className;
       }
-      if (className.equals(FIFOCompactionPolicy.class.getName()) == false) {
+      if (!compactionPolicy.equals(FIFOCompactionPolicy.class.getName())) {
         continue;
       }
       // FIFOCompaction


[14/50] [abbrv] hbase git commit: HBASE-15038 ExportSnapshot should support separate configs for source and destination

Posted by jm...@apache.org.
HBASE-15038 ExportSnapshot should support separate configs for source and destination


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

Branch: refs/heads/trunk
Commit: 9589a7d8be4d29bffcb0c711e5bd6573e8df712c
Parents: a82f7fc
Author: Gary Helmling <ga...@apache.org>
Authored: Wed Dec 23 18:49:58 2015 -0800
Committer: Gary Helmling <ga...@apache.org>
Committed: Mon Jan 4 00:10:19 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/snapshot/ExportSnapshot.java   | 33 ++++++++++++++------
 1 file changed, 23 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9589a7d8/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index beddce3..a574410 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -89,6 +89,10 @@ import org.apache.hadoop.util.ToolRunner;
 @InterfaceStability.Evolving
 public class ExportSnapshot extends Configured implements Tool {
   public static final String NAME = "exportsnapshot";
+  /** Configuration prefix for overrides for the source filesystem */
+  public static final String CONF_SOURCE_PREFIX = NAME + ".from.";
+  /** Configuration prefix for overrides for the destination filesystem */
+  public static final String CONF_DEST_PREFIX = NAME + ".to.";
 
   private static final Log LOG = LogFactory.getLog(ExportSnapshot.class);
 
@@ -141,6 +145,9 @@ public class ExportSnapshot extends Configured implements Tool {
     @Override
     public void setup(Context context) throws IOException {
       Configuration conf = context.getConfiguration();
+      Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX);
+      Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX);
+
       verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true);
 
       filesGroup = conf.get(CONF_FILES_GROUP);
@@ -155,15 +162,15 @@ public class ExportSnapshot extends Configured implements Tool {
       testFailures = conf.getBoolean(CONF_TEST_FAILURE, false);
 
       try {
-        conf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
-        inputFs = FileSystem.get(inputRoot.toUri(), conf);
+        srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
+        inputFs = FileSystem.get(inputRoot.toUri(), srcConf);
       } catch (IOException e) {
         throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e);
       }
 
       try {
-        conf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true);
-        outputFs = FileSystem.get(outputRoot.toUri(), conf);
+        destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true);
+        outputFs = FileSystem.get(outputRoot.toUri(), destConf);
       } catch (IOException e) {
         throw new IOException("Could not get the output FileSystem with root="+ outputRoot, e);
       }
@@ -789,8 +796,12 @@ public class ExportSnapshot extends Configured implements Tool {
     job.setNumReduceTasks(0);
 
     // Acquire the delegation Tokens
+    Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX);
+    TokenCache.obtainTokensForNamenodes(job.getCredentials(),
+      new Path[] { inputRoot }, srcConf);
+    Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX);
     TokenCache.obtainTokensForNamenodes(job.getCredentials(),
-      new Path[] { inputRoot, outputRoot }, conf);
+        new Path[] { outputRoot }, destConf);
 
     // Run the MR Job
     if (!job.waitForCompletion(true)) {
@@ -913,11 +924,13 @@ public class ExportSnapshot extends Configured implements Tool {
       targetName = snapshotName;
     }
 
-    conf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
-    FileSystem inputFs = FileSystem.get(inputRoot.toUri(), conf);
+    Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX);
+    srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true);
+    FileSystem inputFs = FileSystem.get(inputRoot.toUri(), srcConf);
     LOG.debug("inputFs=" + inputFs.getUri().toString() + " inputRoot=" + inputRoot);
-    conf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true);
-    FileSystem outputFs = FileSystem.get(outputRoot.toUri(), conf);
+    Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX);
+    destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true);
+    FileSystem outputFs = FileSystem.get(outputRoot.toUri(), destConf);
     LOG.debug("outputFs=" + outputFs.getUri().toString() + " outputRoot=" + outputRoot.toString());
 
     boolean skipTmp = conf.getBoolean(CONF_SKIP_TMP, false);
@@ -1004,7 +1017,7 @@ public class ExportSnapshot extends Configured implements Tool {
       // Step 4 - Verify snapshot integrity
       if (verifyTarget) {
         LOG.info("Verify snapshot integrity");
-        verifySnapshot(conf, outputFs, outputRoot, outputSnapshotDir);
+        verifySnapshot(destConf, outputFs, outputRoot, outputSnapshotDir);
       }
 
       LOG.info("Export Completed: " + targetName);


[16/50] [abbrv] hbase git commit: HBASE-14902 Revert some of the stringency recently introduced by checkstyle tightening

Posted by jm...@apache.org.
HBASE-14902 Revert some of the stringency recently introduced by checkstyle tightening


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

Branch: refs/heads/trunk
Commit: 998b9371c9d9a6fe4a93e734ba0055c3e20e0b3e
Parents: 0065668
Author: stack <st...@apache.org>
Authored: Mon Jan 4 14:21:59 2016 -0800
Committer: stack <st...@apache.org>
Committed: Mon Jan 4 14:22:12 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/998b9371/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
index e7272c5..6670182 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
@@ -41,7 +41,9 @@
     http://checkstyle.sourceforge.net/config_blocks.html -->
     <module name="EmptyBlock"/>
     <module name="LeftCurly"/>
-    <module name="NeedBraces"/>
+    <module name="NeedBraces">
+      <property name="allowSingleLineStatement" value="true"/>
+    </module>
 
     <!-- Class Design Checks
     http://checkstyle.sourceforge.net/config_design.html -->
@@ -77,7 +79,9 @@
 
     <!-- Javadoc Checks
     http://checkstyle.sourceforge.net/config_javadoc.html -->
-    <module name="JavadocTagContinuationIndentation"/>
+    <module name="JavadocTagContinuationIndentation">
+      <property name="offset" value="2"/>
+    </module>
     <module name="NonEmptyAtclauseDescription"/>
 
     <!-- Miscellaneous Checks


[37/50] [abbrv] hbase git commit: HBASE-13525 replace test-patch with customizations needed to use yetus

Posted by jm...@apache.org.
HBASE-13525 replace test-patch with customizations needed to use yetus

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

Branch: refs/heads/trunk
Commit: 164aeb53992150f0336f17a32ecb2fb733495964
Parents: 28c2b18
Author: Sean Busbey <bu...@apache.org>
Authored: Sun Dec 27 23:26:49 2015 -0600
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Jan 8 07:39:40 2016 -0600

----------------------------------------------------------------------
 dev-support/hbase-personality.sh  |  345 ++++++++++
 dev-support/test-patch.properties |   35 --
 dev-support/test-patch.sh         | 1070 --------------------------------
 pom.xml                           |    3 +
 4 files changed, 348 insertions(+), 1105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
new file mode 100755
index 0000000..2d31fd5
--- /dev/null
+++ b/dev-support/hbase-personality.sh
@@ -0,0 +1,345 @@
+#!/usr/bin/env bash
+# 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.
+
+# You'll need a local installation of
+# [Apache Yetus' precommit checker](http://yetus.apache.org/documentation/0.1.0/#yetus-precommit)
+# to use this personality.
+#
+# Download from: http://yetus.apache.org/downloads/ . You can either grab the source artifact and
+# build from it, or use the convenience binaries provided on that download page.
+#
+# To run against, e.g. HBASE-15074 you'd then do
+# ```bash
+# test-patch --personality=dev-support/hbase-personality.sh HBASE-15074
+# ```
+#
+# If you want to skip the ~1 hour it'll take to do all the hadoop API checks, use
+# ```bash
+# test-patch  --plugins=all,-hadoopcheck --personality=dev-support/hbase-personality.sh HBASE-15074
+# ````
+#
+# pass the `--jenkins` flag if you want to allow test-patch to destructively alter local working
+# directory / branch in order to have things match what the issue patch requests.
+
+personality_plugins "all"
+
+function personality_globals
+{
+  #shellcheck disable=SC2034
+  PROJECT_NAME=hbase
+  #shellcheck disable=SC2034
+  PATCH_BRANCH_DEFAULT=master
+  #shellcheck disable=SC2034
+  JIRA_ISSUE_RE='^HBASE-[0-9]+$'
+  #shellcheck disable=SC2034
+  GITHUB_REPO="apache/hbase"
+
+  # TODO use PATCH_BRANCH to select hadoop versions to use.
+  # All supported Hadoop versions that we want to test the compilation with
+  HBASE_HADOOP_VERSIONS="2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.1 2.6.2 2.6.3 2.7.1"
+
+  # TODO use PATCH_BRANCH to select jdk versions to use.
+
+  # Override the maven options
+  MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
+
+}
+
+function personality_modules
+{
+  local repostatus=$1
+  local testtype=$2
+  local extra=""
+
+  yetus_debug "Personality: ${repostatus} ${testtype}"
+
+  clear_personality_queue
+
+  extra="-DHBasePatchProcess"
+
+  if [[ ${repostatus} == branch
+     && ${testtype} == mvninstall ]];then
+     personality_enqueue_module . ${extra}
+     return
+  fi
+
+  if [[ ${testtype} = findbugs ]]; then
+    for module in ${CHANGED_MODULES}; do
+      # skip findbugs on hbase-shell
+      if [[ ${module} == hbase-shell ]]; then
+        continue
+      else
+        # shellcheck disable=SC2086
+        personality_enqueue_module ${module} ${extra}
+      fi
+    done
+    return
+  fi
+
+  if [[ ${testtype} = unit ]]; then
+    extra="${extra} -PrunAllTests"
+
+    # Inject the jenkins build-id for our surefire invocations
+    # Used by zombie detection stuff, even though we're not including that yet.
+    if [ -n "${BUILD_ID}" ]; then
+      extra="${extra} -Dbuild.id=${BUILD_ID}"
+    fi
+  fi
+
+  for module in ${CHANGED_MODULES}; do
+    # shellcheck disable=SC2086
+    personality_enqueue_module ${module} ${extra}
+  done
+}
+
+###################################################
+# Below here are our one-off tests specific to hbase.
+# TODO break them into individual files so it's easier to maintain them?
+
+# TODO line length check? could ignore all java files since checkstyle gets them.
+
+###################################################
+
+add_test_type hadoopcheck
+
+function hadoopcheck_filefilter
+{
+  local filename=$1
+
+  if [[ ${filename} =~ \.java$ ]]; then
+    add_test hadoopcheck
+  fi
+}
+
+function hadoopcheck_rebuild
+{
+  local repostatus=$1
+  local hadoopver
+  local logfile
+  local count
+  local result=0
+
+  if [[ "${repostatus}" = branch ]]; then
+    return 0
+  fi
+
+  big_console_header "Compiling against various Hadoop versions"
+
+  export MAVEN_OPTS="${MAVEN_OPTS}"
+  for hadoopver in ${HBASE_HADOOP_VERSIONS}; do
+    logfile="${PATCH_DIR}/patch-javac-${hadoopver}.txt"
+    echo_and_redirect "${logfile}" \
+      "${MAVEN}" clean install \
+        -DskipTests -DHBasePatchProcess \
+        -Dhadoop-two.version="${hadoopver}"
+    count=$(${GREP} -c ERROR "${logfile}")
+    if [[ ${count} -gt 0 ]]; then
+      add_vote_table -1 hadoopcheck "Patch causes ${count} errors with Hadoop v${hadoopver}."
+      ((result=result+1))
+    fi
+  done
+
+  if [[ ${result} -gt 0 ]]; then
+    return 1
+  fi
+
+  add_vote_table +1 hadoopcheck "Patch does not cause any errors with Hadoop ${HBASE_HADOOP_VERSIONS}."
+  return 0
+}
+
+######################################
+
+# TODO if we need th protoc check, we probably need to check building all the modules that rely on hbase-protocol
+add_test_type hbaseprotoc
+
+function hbaseprotoc_filefilter
+{
+  local filename=$1
+
+  if [[ ${filename} =~ \.proto$ ]]; then
+    add_test hbaseprotoc
+  fi
+}
+
+function hbaseprotoc_rebuild
+{
+  local i=0
+  local fn
+  local module
+  local logfile
+  local count
+  local result
+
+  if [[ "${repostatus}" = branch ]]; then
+    return 0
+  fi
+
+  verify_needed_test hbaseprotoc
+  if [[ $? == 0 ]]; then
+    return 0
+  fi
+
+  big_console_header "Patch HBase protoc plugin"
+
+  start_clock
+
+
+  personality_modules patch hbaseprotoc
+  modules_workers patch hbaseprotoc compile -DskipTests -Pcompile-protobuf -X -DHBasePatchProcess
+
+  # shellcheck disable=SC2153
+  until [[ $i -eq ${#MODULE[@]} ]]; do
+    if [[ ${MODULE_STATUS[${i}]} == -1 ]]; then
+      ((result=result+1))
+      ((i=i+1))
+      continue
+    fi
+    module=${MODULE[$i]}
+    fn=$(module_file_fragment "${module}")
+    logfile="${PATCH_DIR}/patch-hbaseprotoc-${fn}.txt"
+
+    count=$(${GREP} -c ERROR "${logfile}")
+
+    if [[ ${count} -gt 0 ]]; then
+      module_status ${i} -1 "patch-hbaseprotoc-${fn}.txt" "Patch generated "\
+        "${count} new protoc errors in ${module}."
+      ((result=result+1))
+    fi
+    ((i=i+1))
+  done
+
+  modules_messages patch hbaseprotoc true
+  if [[ ${result} -gt 0 ]]; then
+    return 1
+  fi
+  return 0
+}
+
+######################################
+
+add_test_type hbaseanti
+
+function hbaseanti_filefilter
+{
+  local filename=$1
+
+  if [[ ${filename} =~ \.java$ ]]; then
+    add_test hbaseanti
+  fi
+}
+
+function hbaseanti_patchfile
+{
+  local patchfile=$1
+  local warnings
+  local result
+
+  verify_needed_test hbaseanti
+  if [[ $? == 0 ]]; then
+    return 0
+  fi
+
+  big_console_header "Checking for known anti-patterns"
+
+  start_clock
+
+  warnings=$(${GREP} 'new TreeMap<byte.*()' "${patchfile}")
+  if [[ ${warnings} -gt 0 ]]; then
+    add_vote_table -1 hbaseanti "" "The patch appears to have anti-pattern where BYTES_COMPARATOR was omitted: ${warnings}."
+    ((result=result+1))
+  fi
+
+  warnings=$(${GREP} 'import org.apache.hadoop.classification' "${patchfile}")
+  if [[ ${warnings} -gt 0 ]]; then
+    add_vote_table -1 hbaseanti "" "The patch appears use Hadoop classification instead of HBase: ${warnings}."
+    ((result=result+1))
+  fi
+
+  if [[ ${result} -gt 0 ]]; then
+    return 1
+  fi
+
+  add_vote_table +1 hbaseanti "" "Patch does not have any anti-patterns."
+  return 0
+}
+
+# Work around HBASE-15042
+function mvnsite_filefilter
+{
+  local filename=$1
+
+  if [[ ${BUILDTOOL} = maven ]]; then
+    if [[ ${filename} =~ src/main/site || ${filename} =~ src/main/asciidoc ]]; then
+      yetus_debug "tests/mvnsite: ${filename}"
+      add_test mvnsite
+    fi
+  fi
+}
+
+## This is named so that yetus will check us right after running tests.
+## Essentially, we check for normal failures and then we look for zombies.
+#function hbase_unit_logfilter
+#{
+#  declare testtype="unit"
+#  declare input=$1
+#  declare output=$2
+#  declare processes
+#  declare process_output
+#  declare zombies
+#  declare zombie_count=0
+#  declare zombie_process
+#
+#  yetus_debug "in hbase-specific unit logfilter."
+#
+#  # pass-through to whatever is counting actual failures
+#  if declare -f ${BUILDTOOL}_${testtype}_logfilter >/dev/null; then
+#    "${BUILDTOOL}_${testtype}_logfilter" "${input}" "${output}"
+#  elif declare -f ${testtype}_logfilter >/dev/null; then
+#    "${testtype}_logfilter" "${input}" "${output}"
+#  fi
+#
+#  start_clock
+#  if [ -n "${BUILD_ID}" ]; then
+#    yetus_debug "Checking for zombie test processes."
+#    processes=$(jps -v | "${GREP}" surefirebooter | "${GREP}" -e "hbase.build.id=${BUILD_ID}")
+#    if [ -n "${processes}" ] && [ "$(echo "${processes}" | wc -l)" -gt 0 ]; then
+#      yetus_warn "Found some suspicious process(es). Waiting a bit to see if they're just slow to stop."
+#      yetus_debug "${processes}"
+#      sleep 30
+#      #shellcheck disable=SC2016
+#      for pid in $(echo "${processes}"| ${AWK} '{print $1}'); do
+#        # Test our zombie still running (and that it still an hbase build item)
+#        process_output=$(ps -p "${pid}" | tail +2 | "${GREP}" -e "hbase.build.id=${BUILD_ID}")
+#        if [[ -n "${process_output}" ]]; then
+#          yetus_error "Zombie: ${process_output}"
+#          ((zombie_count = zombie_count + 1))
+#          zombie_process=$(jstack "${pid}" | "${GREP}" -e "\.Test" | "${GREP}" -e "\.java"| head -3)
+#          zombies="${zombies} ${zombie_process}"
+#        fi
+#      done
+#    fi
+#    if [ "${zombie_count}" -ne 0 ]; then
+#      add_vote_table -1 zombies "There are ${zombie_count} zombie test(s)"
+#      populate_test_table "zombie unit tests" "${zombies}"
+#    else
+#      yetus_info "Zombie check complete. All test runs exited normally."
+#      stop_clock
+#    fi
+#  else
+#    add_vote_table -0 zombies "There is no BUILD_ID env variable; can't check for zombies."
+#  fi
+#
+#}

http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/dev-support/test-patch.properties
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.properties b/dev-support/test-patch.properties
deleted file mode 100644
index bc29896..0000000
--- a/dev-support/test-patch.properties
+++ /dev/null
@@ -1,35 +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.
-
-MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
-
-# The number of acceptable warning for *all* modules
-# Please update the per-module test-patch.properties if you update this file.
-
-OK_RELEASEAUDIT_WARNINGS=0
-# Allow four warnings.  Javadoc complains about sun.misc.Unsafe use.
-# See HBASE-7457, HBASE-13761
-# Allow 2 additional warnings for Scala stub notice about MR. See HBASE-13992
-OK_JAVADOC_WARNINGS=9
-
-MAX_LINE_LENGTH=100
-
-# All supported branches for testing with precommit build
-# branch-1.x should apprear before branch-1 since the latter is a prefix
-BRANCH_NAMES="0.94 0.98 branch-1.0 branch-1.1 branch-1.2 branch-1 master hbase-12439 hbase-11339"
-
-# All supported Hadoop versions that we want to test the compilation with
-HADOOP2_VERSIONS="2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.0 2.6.1 2.7.0 2.7.1"
-HADOOP3_VERSIONS="3.0.0-SNAPSHOT"

http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
deleted file mode 100755
index d0c0346..0000000
--- a/dev-support/test-patch.sh
+++ /dev/null
@@ -1,1070 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-
-
-#set -x
-
-### Setup some variables.  
-### GIT_COMMIT and BUILD_URL are set by Hudson if it is run by patch process
-### Read variables from properties file
-bindir=$(dirname $0)
-
-# Defaults
-if [ -z "$MAVEN_HOME" ]; then
-  MVN=mvn
-else
-  MVN=$MAVEN_HOME/bin/mvn
-fi
-
-NEWLINE=$'\n'
-
-PROJECT_NAME=HBase
-JENKINS=false
-MOVE_PATCH_DIR=true
-PATCH_DIR=/tmp
-BASEDIR=$(pwd)
-BRANCH_NAME="master"
-
-. $BASEDIR/dev-support/test-patch.properties
-
-PS=${PS:-ps}
-AWK=${AWK:-awk}
-WGET=${WGET:-wget}
-GREP=${GREP:-grep}
-EGREP=${EGREP:-egrep}
-PATCH=${PATCH:-patch}
-JIRACLI=${JIRA:-jira}
-FINDBUGS_HOME=${FINDBUGS_HOME}
-FORREST_HOME=${FORREST_HOME}
-ECLIPSE_HOME=${ECLIPSE_HOME}
-GIT=${GIT:-git}
-
-###############################################################################
-printUsage() {
-  echo "Usage: $0 [options] patch-file | defect-number"
-  echo
-  echo "Where:"
-  echo "  patch-file is a local patch file containing the changes to test"
-  echo "  defect-number is a JIRA defect number (e.g. 'HADOOP-1234') to test (Jenkins only)"
-  echo
-  echo "Options:"
-  echo "--patch-dir=<dir>      The directory for working and output files (default '/tmp')"
-  echo "--basedir=<dir>        The directory to apply the patch to (default current directory)"
-  echo "--mvn-cmd=<cmd>        The 'mvn' command to use (default \$MAVEN_HOME/bin/mvn, or 'mvn')"
-  echo "--ps-cmd=<cmd>         The 'ps' command to use (default 'ps')"
-  echo "--awk-cmd=<cmd>        The 'awk' command to use (default 'awk')"
-  echo "--grep-cmd=<cmd>       The 'grep' command to use (default 'grep')"
-  echo "--patch-cmd=<cmd>      The 'patch' command to use (default 'patch')"
-  echo "--findbugs-home=<path> Findbugs home directory (default FINDBUGS_HOME environment variable)"
-  echo "--forrest-home=<path>  Forrest home directory (default FORREST_HOME environment variable)"
-  echo "--dirty-workspace      Allow the local workspace to have uncommitted changes"
-  echo "--git-cmd=<cmd>        The 'git' command to use (default 'git')"
-  echo
-  echo "Jenkins-only options:"
-  echo "--jenkins              Run by Jenkins (runs tests and posts results to JIRA)"
-  echo "--wget-cmd=<cmd>       The 'wget' command to use (default 'wget')"
-  echo "--jira-cmd=<cmd>       The 'jira' command to use (default 'jira')"
-  echo "--jira-password=<pw>   The password for the 'jira' command"
-  echo "--eclipse-home=<path>  Eclipse home directory (default ECLIPSE_HOME environment variable)"
-}
-
-###############################################################################
-parseArgs() {
-  for i in $*
-  do
-    case $i in
-    --jenkins)
-      JENKINS=true
-      ;;
-    --no-move-patch-dir)
-      MOVE_PATCH_DIR=false
-      ;;
-    --patch-dir=*)
-      PATCH_DIR=${i#*=}
-      ;;
-    --basedir=*)
-      BASEDIR=${i#*=}
-      ;;
-    --mvn-cmd=*)
-      MVN=${i#*=}
-      ;;
-    --ps-cmd=*)
-      PS=${i#*=}
-      ;;
-    --awk-cmd=*)
-      AWK=${i#*=}
-      ;;
-    --wget-cmd=*)
-      WGET=${i#*=}
-      ;;
-    --grep-cmd=*)
-      GREP=${i#*=}
-      ;;
-    --patch-cmd=*)
-      PATCH=${i#*=}
-      ;;
-    --jira-cmd=*)
-      JIRACLI=${i#*=}
-      ;;
-    --jira-password=*)
-      JIRA_PASSWD=${i#*=}
-      ;;
-    --findbugs-home=*)
-      FINDBUGS_HOME=${i#*=}
-      ;;
-    --forrest-home=*)
-      FORREST_HOME=${i#*=}
-      ;;
-    --eclipse-home=*)
-      ECLIPSE_HOME=${i#*=}
-      ;;
-    --dirty-workspace)
-      DIRTY_WORKSPACE=true
-      ;;
-    --git-cmd=*)
-      GIT=${i#*=}
-      ;;
-    *)
-      PATCH_OR_DEFECT=$i
-      ;;
-    esac
-  done
-  if [ -z "$PATCH_OR_DEFECT" ]; then
-    printUsage
-    exit 1
-  fi
-  if [[ $JENKINS == "true" ]] ; then
-    echo "Running in Jenkins mode"
-    defect=$PATCH_OR_DEFECT
-    ECLIPSE_PROPERTY="-Declipse.home=$ECLIPSE_HOME"
-  else
-    echo "Running in developer mode"
-    JENKINS=false
-    ### PATCH_FILE contains the location of the patchfile
-    PATCH_FILE=$PATCH_OR_DEFECT
-    if [[ ! -e "$PATCH_FILE" ]] ; then
-      echo "Unable to locate the patch file $PATCH_FILE"
-      cleanupAndExit 0
-    fi
-    ### Check if $PATCH_DIR exists. If it does not exist, create a new directory
-    if [[ ! -e "$PATCH_DIR" ]] ; then
-      mkdir "$PATCH_DIR"
-      if [[ $? == 0 ]] ; then 
-        echo "$PATCH_DIR has been created"
-      else
-        echo "Unable to create $PATCH_DIR"
-        cleanupAndExit 0
-      fi
-    fi
-    ### Obtain the patch filename to append it to the version number
-    defect=`basename $PATCH_FILE`
-  fi
-}
-
-###############################################################################
-checkout () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Testing patch for ${defect}."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  ### When run by a developer, if the workspace contains modifications, do not continue
-  ### unless the --dirty-workspace option was set
-  if [[ $JENKINS == "false" ]] ; then
-    if [[ -z $DIRTY_WORKSPACE ]] ; then
-      # Ref http://stackoverflow.com/a/2659808 for details on checking dirty status
-      ${GIT} diff-index --quiet HEAD
-      if [[ $? -ne 0 ]] ; then
-        uncommitted=`${GIT} diff --name-only HEAD`
-        uncommitted="You have the following files with uncommitted changes:${NEWLINE}${uncommitted}"
-      fi
-      untracked="$(${GIT} ls-files --exclude-standard --others)" && test -z "${untracked}"
-      if [[ $? -ne 0 ]] ; then
-        untracked="You have untracked and unignored files:${NEWLINE}${untracked}"
-      fi
-      if [[ $uncommitted || $untracked ]] ; then
-        echo "ERROR: can't run in a workspace that contains modifications."
-        echo "Pass the '--dirty-workspace' flag to bypass."
-        echo ""
-        echo "${uncommitted}"
-        echo ""
-        echo "${untracked}"
-        cleanupAndExit 1
-      fi
-    fi
-    echo
-  fi
-  return $?
-}
-
-findBranchNameFromPatchName() {
-  local patchName=$1
-  for LOCAL_BRANCH_NAME in $BRANCH_NAMES; do
-    if [[ $patchName =~ /jira/secure/attachment/[0-9]*/.*$LOCAL_BRANCH_NAME ]]; then
-      BRANCH_NAME=$LOCAL_BRANCH_NAME
-      break
-    fi
-  done
-  return 0
-}
-
-checkoutBranch() {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Testing patch on branch ${BRANCH_NAME}."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  if [[ $JENKINS == "true" ]] ; then
-    if [[ "$BRANCH_NAME" != "master" ]]; then
-      echo "origin/${BRANCH_NAME} HEAD is commit `${GIT} rev-list origin/${BRANCH_NAME} -1`"
-      echo "${GIT} checkout -f  `${GIT} rev-list origin/${BRANCH_NAME} -1`"
-      ${GIT} checkout -f  `${GIT} rev-list origin/${BRANCH_NAME} -1`
-      echo "${GIT} status"
-      ${GIT} status
-    fi
-  fi
-}
-
-###############################################################################
-###  Collect findbugs reports
-collectFindbugsReports() {
-  name=$1
-  basedir=$2
-  patch_dir=$3
-  for file in $(find $basedir -name findbugsXml.xml)
-  do
-    relative_file=${file#$basedir/} # strip leading $basedir prefix
-    if [ ! $relative_file == "target/findbugsXml.xml" ]; then
-      module_suffix=${relative_file%/target/findbugsXml.xml} # strip trailing path
-      module_suffix=`basename ${module_suffix}`
-    fi
-
-    cp $file $patch_dir/${name}FindbugsWarnings${module_suffix}.xml
-    $FINDBUGS_HOME/bin/setBugDatabaseInfo -name $name \
-      $patch_dir/${name}FindbugsWarnings${module_suffix}.xml \
-      $patch_dir/${name}FindbugsWarnings${module_suffix}.xml
-  done
-  xml_file=$patch_dir/${name}FindbugsWarnings.xml
-  html_file=$patch_dir/${name}FindbugsWarnings.html
-  $FINDBUGS_HOME/bin/unionBugs -withMessages \
-	 -output $xml_file $patch_dir/${name}FindbugsWarnings*.xml
-  $FINDBUGS_HOME/bin/convertXmlToText -html $xml_file $html_file
-  file $xml_file $html_file
-}
-
-###############################################################################
-setup () {
-  ### Download latest patch file (ignoring .htm and .html) when run from patch process
-  if [[ $JENKINS == "true" ]] ; then
-    $WGET -q -O $PATCH_DIR/jira http://issues.apache.org/jira/browse/$defect
-    if [[ `$GREP -c 'Patch Available' $PATCH_DIR/jira` == 0 ]] ; then
-      echo "$defect is not \"Patch Available\".  Exiting."
-      cleanupAndExit 0
-    fi
-    relativePatchURL=`$GREP -o '"/jira/secure/attachment/[0-9]*/[^"]*' $PATCH_DIR/jira | $EGREP '(\.txt$|\.patch$|\.diff$)' | sort | tail -1 | $GREP -o '/jira/secure/attachment/[0-9]*/[^"]*'`
-    patchURL="http://issues.apache.org${relativePatchURL}"
-    patchNum=`echo $patchURL | $GREP -o '[0-9]*/' | $GREP -o '[0-9]*'`
-    # ensure attachment has not already been tested
-    ATTACHMENT_ID=$(basename $(dirname $patchURL))
-    if grep -q "ATTACHMENT ID: $ATTACHMENT_ID" $PATCH_DIR/jira
-    then
-      echo "Attachment $ATTACHMENT_ID is already tested for $defect"
-      exit 1
-    fi
-    echo "$defect patch is being downloaded at `date` from"
-    echo "$patchURL"
-    $WGET -q -O $PATCH_DIR/patch $patchURL
-    VERSION=${GIT_COMMIT}_${defect}_PATCH-${patchNum}
-    findBranchNameFromPatchName ${relativePatchURL}
-    checkoutBranch
-    JIRA_COMMENT="Here are the results of testing the latest attachment 
-  $patchURL
-  against ${BRANCH_NAME} branch at commit ${GIT_COMMIT}.
-  ATTACHMENT ID: ${ATTACHMENT_ID}"
-
-  ### Copy the patch file to $PATCH_DIR
-  else
-    VERSION=PATCH-${defect}
-    cp $PATCH_FILE $PATCH_DIR/patch
-    if [[ $? == 0 ]] ; then
-      echo "Patch file $PATCH_FILE copied to $PATCH_DIR"
-    else
-      echo "Could not copy $PATCH_FILE to $PATCH_DIR"
-      cleanupAndExit 0
-    fi
-  fi
-  ### exit if warnings are NOT defined in the properties file
-  if [[ -z "$OK_JAVADOC_WARNINGS" ]] || [[ -z $OK_RELEASEAUDIT_WARNINGS ]] ; then
-    echo "Please define the following properties in test-patch.properties file"
-	 echo  "OK_RELEASEAUDIT_WARNINGS"
-	 echo  "OK_JAVADOC_WARNINGS"
-    cleanupAndExit 1
-  fi
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo " Pre-build master to verify stability and javac warnings"
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package checkstyle:checkstyle-aggregate findbugs:findbugs -DskipTests \
- -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  # build core and tests
-  $MVN clean package checkstyle:checkstyle-aggregate findbugs:findbugs -DskipTests \
-	 -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1
-  if [[ $? != 0 ]] ; then
-    echo "mvn exit code was $?"
-    ERR=`$GREP -A 5 'Compilation failure' $PATCH_DIR/trunkJavacWarnings.txt`
-    if [[ ${#ERR} -ge 1 ]] ; then
-      echo "Trunk compilation is broken?
-      {code}$ERR{code}"
-      cleanupAndExit 1
-    fi
-  fi
-  mv target/checkstyle-result.xml $PATCH_DIR/trunkCheckstyle.xml
-  collectFindbugsReports trunk $BASEDIR $PATCH_DIR
-}
-
-###############################################################################
-### Check for @author tags in the patch
-checkAuthor () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking there are no @author tags in the patch."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  authorTags=`$GREP -c -i '@author' $PATCH_DIR/patch`
-  echo "There appear to be $authorTags @author tags in the patch."
-  if [[ $authorTags != 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 @author{color}.  The patch appears to contain $authorTags @author tags which the Hadoop community has agreed to not allow in code contributions."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 @author{color}.  The patch does not contain any @author tags."
-  return 0
-}
-
-###############################################################################
-### Check for tests in the patch
-checkTests () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking there are new or changed tests in the patch."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  testReferences=`$GREP -c -i '/test' $PATCH_DIR/patch`
-  echo "There appear to be $testReferences test files referenced in the patch."
-  if [[ $testReferences == 0 ]] ; then
-    if [[ $JENKINS == "true" ]] ; then
-      patchIsDoc=`$GREP -c -i 'title="documentation' $PATCH_DIR/jira`
-      if [[ $patchIsDoc != 0 ]] ; then
-        echo "The patch appears to be a documentation patch that doesn't require tests."
-        JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+0 tests included{color}.  The patch appears to be a documentation patch that doesn't require tests."
-        return 0
-      fi
-    fi
-    srcReferences=`${GREP} "diff --git" "${PATCH_DIR}/patch" | ${GREP} "src/main" | \
-        ${GREP} -v "src/main/asciidoc" | ${GREP} -v "src/main/site" -c`
-    if [[ $srcReferences == 0 ]] ; then
-      echo "The patch doesn't appear to alter any code that requires tests."
-      JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+0 tests included{color}.  The patch appears to be a documentation, build,
-                        or dev-support patch that doesn't require tests."
-      return 0
-    fi
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 tests included{color}.  The patch doesn't appear to include any new or modified tests.
-                        Please justify why no new tests are needed for this patch.
-                        Also please list what manual steps were performed to verify this patch."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 tests included{color}.  The patch appears to include $testReferences new or modified tests."
-  return 0
-}
-
-###############################################################################
-### Check there are no compilation errors, passing a file to be parsed.
-checkCompilationErrors() {
-  local file=$1
-  hadoopVersion=""
-  if [ "$#" -ne 1 ]; then
-    hadoopVersion="with Hadoop version $2"
-  fi
-  COMPILATION_ERROR=false
-  eval $(awk '/ERROR/ {print "COMPILATION_ERROR=true"}' $file)
-  if $COMPILATION_ERROR ; then
-    ERRORS=$($AWK '/ERROR/ { print $0 }' $file)
-    echo "======================================================================"
-    echo "There are compilation errors $hadoopVersion."
-    echo "======================================================================"
-    echo "$ERRORS"
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javac{color}.  The patch appears to cause mvn compile goal to fail $hadoopVersion.
-
-    Compilation errors resume:
-    $ERRORS
-    "
-    submitJiraComment 1
-    cleanupAndExit 1
-  fi
-}
-
-###############################################################################
-### Check there are no protoc compilation errors, passing a file to be parsed.
-checkProtocCompilationErrors() {
-  local file=$1
-  COMPILATION_ERROR=false
-  eval $(awk '/\[ERROR/ {print "COMPILATION_ERROR=true"}' $file)
-  if $COMPILATION_ERROR ; then
-    ERRORS=$($AWK '/\[ERROR/ { print $0 }' $file)
-    echo "======================================================================"
-    echo "There are Protoc compilation errors."
-    echo "======================================================================"
-    echo "$ERRORS"
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javac{color}.  The patch appears to cause mvn compile-protobuf profile to fail.
-
-    Protoc Compilation errors resume:
-    $ERRORS
-    "
-    cleanupAndExit 1
-  fi
-}
-
-###############################################################################
-### Attempt to apply the patch
-applyPatch () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Applying patch."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
- 
-  export PATCH
-  $BASEDIR/dev-support/smart-apply-patch.sh $PATCH_DIR/patch
-  if [[ $? != 0 ]] ; then
-    echo "PATCH APPLICATION FAILED"
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 patch{color}.  The patch command could not apply the patch."
-    return 1
-  fi
-  return 0
-}
-
-###############################################################################
-### Check against known anti-patterns
-checkAntiPatterns () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking against known anti-patterns."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  warnings=`$GREP 'new TreeMap<byte.*()' $PATCH_DIR/patch`
-  if [[ $warnings != "" ]]; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 Anti-pattern{color}.  The patch appears to have anti-pattern where BYTES_COMPARATOR was omitted: $warnings."
-    return 1
-  fi
-  return 0
-}
-
-###############################################################################
-### Check that there are no incorrect annotations
-checkInterfaceAudience () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking against hadoop InterfaceAudience."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  warnings=`$GREP 'import org.apache.hadoop.classification' $PATCH_DIR/patch`
-  if [[ $warnings != "" ]]; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 InterfaceAudience{color}.  The patch appears to contain InterfaceAudience from hadoop rather than hbase: $warnings."
-    return 1
-  fi
-  return 0
-}
-
-###############################################################################
-### Check there are no javadoc warnings
-checkJavadocWarnings () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched javadoc warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package javadoc:javadoc -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavadocWarnings.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean package javadoc:javadoc -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavadocWarnings.txt 2>&1
-  javadocWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/patchJavadocWarnings.txt | $AWK '/Javadoc Warnings/,EOF' | $GREP warning | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
-  echo ""
-  echo ""
-  echo "There appear to be $javadocWarnings javadoc warnings generated by the patched build."
-
-  ### if current warnings greater than OK_JAVADOC_WARNINGS
-  if [[ $javadocWarnings -gt $OK_JAVADOC_WARNINGS ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javadoc{color}.  The javadoc tool appears to have generated `expr $(($javadocWarnings-$OK_JAVADOC_WARNINGS))` warning messages."
-    # Add javadoc output url
-    JIRA_COMMENT_FOOTER="Javadoc warnings: $BUILD_URL/artifact/patchprocess/patchJavadocWarnings.txt
-$JIRA_COMMENT_FOOTER"
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 javadoc{color}.  The javadoc tool did not generate any warning messages."
-  return 0
-}
-
-checkBuildWithHadoopVersions() {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Building with all supported Hadoop versions ."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  for HADOOP2_VERSION in $HADOOP2_VERSIONS ; do
-    echo "$MVN clean install -DskipTests -D${PROJECT_NAME}PatchProcess -Dhadoop-two.version=$HADOOP2_VERSION > $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt 2>&1"
-    $MVN clean install -DskipTests -D${PROJECT_NAME}PatchProcess -Dhadoop-two.version=$HADOOP2_VERSION > $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt 2>&1
-    checkCompilationErrors $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt $HADOOP2_VERSION
-  done
-
-  # TODO: add Hadoop3 versions and compilation here when we get the hadoop.profile=3.0 working
-
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 hadoop versions{color}. The patch compiles with all supported hadoop versions ($HADOOP2_VERSIONS)"
-  return 0
-}
-
-###############################################################################
-### Check there are no changes in the number of Javac warnings
-checkJavacWarnings () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched javac warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavacWarnings.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean package -DskipTests -D${PROJECT_NAME}PatchProcess  > $PATCH_DIR/patchJavacWarnings.txt 2>&1
-  checkCompilationErrors $PATCH_DIR/patchJavacWarnings.txt
-  ### Compare trunk and patch javac warning numbers
-  if [[ -f $PATCH_DIR/patchJavacWarnings.txt ]] ; then
-    trunkJavacWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/trunkJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
-    patchJavacWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/patchJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
-    echo "There appear to be $trunkJavacWarnings javac compiler warnings before the patch and $patchJavacWarnings javac compiler warnings after applying the patch."
-    if [[ $patchJavacWarnings != "" && $trunkJavacWarnings != "" ]] ; then
-      if [[ $patchJavacWarnings -gt $trunkJavacWarnings ]] ; then
-        JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javac{color}.  The applied patch generated $patchJavacWarnings javac compiler warnings (more than the master's current $trunkJavacWarnings warnings)."
-        return 1
-      fi
-    fi
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 javac{color}.  The applied patch does not increase the total number of javac compiler warnings."
-  return 0
-}
-
-checkCheckstyleErrors() {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched Checkstyle errors."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  if [[ -f $PATCH_DIR/trunkCheckstyle.xml ]] ; then
-    $MVN package -DskipTests checkstyle:checkstyle-aggregate > /dev/null 2>&1
-    mv target/checkstyle-result.xml $PATCH_DIR/patchCheckstyle.xml
-    mv target/site/checkstyle-aggregate.html $PATCH_DIR
-    mv target/site/checkstyle.css $PATCH_DIR
-    $BASEDIR/dev-support/checkstyle_report.py $PATCH_DIR/trunkCheckstyle.xml $PATCH_DIR/patchCheckstyle.xml
-    if [[ $? -eq 1 ]] ; then
-                JIRA_COMMENT_FOOTER="Checkstyle Errors: $BUILD_URL/artifact/patchprocess/checkstyle-aggregate.html
-
-                $JIRA_COMMENT_FOOTER"
-
-                JIRA_COMMENT="$JIRA_COMMENT
-
-                {color:red}-1 checkstyle{color}.  The applied patch generated new checkstyle errors. Check build console for list of new errors."
-        return 1
-    fi
-  fi
-  JIRA_COMMENT_FOOTER="Checkstyle Errors: $BUILD_URL/artifact/patchprocess/checkstyle-aggregate.html
-
-  $JIRA_COMMENT_FOOTER"
-
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 checkstyle{color}. The applied patch does not generate new checkstyle errors."
-  return 0
-
-}
-###############################################################################
-checkProtocErrors () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining whether there is patched protoc error."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean install -DskipTests -Pcompile-protobuf -X -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchProtocErrors.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean install -DskipTests -Pcompile-protobuf -X -D${PROJECT_NAME}PatchProcess  > $PATCH_DIR/patchProtocErrors.txt 2>&1
-  checkProtocCompilationErrors $PATCH_DIR/patchProtocErrors.txt
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 protoc{color}.  The applied patch does not increase the total number of protoc compiler warnings."
-  return 0
-}
-
-###############################################################################
-### Check there are no changes in the number of release audit (RAT) warnings
-checkReleaseAuditWarnings () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched release audit warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN apache-rat:check -D${PROJECT_NAME}PatchProcess 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN apache-rat:check -D${PROJECT_NAME}PatchProcess 2>&1
-  find $BASEDIR -name rat.txt | xargs cat > $PATCH_DIR/patchReleaseAuditWarnings.txt
-
-  ### Compare trunk and patch release audit warning numbers
-  if [[ -f $PATCH_DIR/patchReleaseAuditWarnings.txt ]] ; then
-    patchReleaseAuditWarnings=`$GREP -c '\!?????' $PATCH_DIR/patchReleaseAuditWarnings.txt`
-    echo ""
-    echo ""
-    echo "There appear to be $OK_RELEASEAUDIT_WARNINGS release audit warnings before the patch and $patchReleaseAuditWarnings release audit warnings after applying the patch."
-    if [[ $patchReleaseAuditWarnings != "" && $OK_RELEASEAUDIT_WARNINGS != "" ]] ; then
-      if [[ $patchReleaseAuditWarnings -gt $OK_RELEASEAUDIT_WARNINGS ]] ; then
-        JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 release audit{color}.  The applied patch generated $patchReleaseAuditWarnings release audit warnings (more than the master's current $OK_RELEASEAUDIT_WARNINGS warnings)."
-        $GREP '\!?????' $PATCH_DIR/patchReleaseAuditWarnings.txt > $PATCH_DIR/patchReleaseAuditProblems.txt
-        echo "Lines that start with ????? in the release audit report indicate files that do not have an Apache license header." >> $PATCH_DIR/patchReleaseAuditProblems.txt
-        JIRA_COMMENT_FOOTER="Release audit warnings: $BUILD_URL/artifact/patchprocess/patchReleaseAuditWarnings.txt
-$JIRA_COMMENT_FOOTER"
-        return 1
-      fi
-    fi
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 release audit{color}.  The applied patch does not increase the total number of release audit warnings."
-  return 0
-}
-
-###############################################################################
-### Check there are no changes in the number of Findbugs warnings
-checkFindbugsWarnings () {
-  findbugs_version=`${FINDBUGS_HOME}/bin/findbugs -version`
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched Findbugs warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package findbugs:findbugs -D${PROJECT_NAME}PatchProcess" 
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean package findbugs:findbugs -D${PROJECT_NAME}PatchProcess -DskipTests < /dev/null
-
-  if [ $? != 0 ] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 findbugs{color}.  The patch appears to cause Findbugs (version ${findbugs_version}) to fail."
-    return 1
-  fi
-
-  collectFindbugsReports patch $BASEDIR $PATCH_DIR 
-  #this files are generated by collectFindbugsReports() named with its first argument
-  patch_xml=$PATCH_DIR/patchFindbugsWarnings.xml
-  trunk_xml=$PATCH_DIR/trunkFindbugsWarnings.xml
-  # combine them to one database
-  combined_xml=$PATCH_DIR/combinedFindbugsWarnings.xml
-  new_xml=$PATCH_DIR/newFindbugsWarnings.xml
-  new_html=$PATCH_DIR/newFindbugsWarnings.html
-  $FINDBUGS_HOME/bin/computeBugHistory -useAnalysisTimes -withMessages \
-	-output $combined_xml $trunk_xml $patch_xml
-  findbugsWarnings=$($FINDBUGS_HOME/bin/filterBugs -first patch $combined_xml $new_xml)
-  findbugsFixedWarnings=$($FINDBUGS_HOME/bin/filterBugs -fixed patch $combined_xml $new_xml)
-  $FINDBUGS_HOME/bin/convertXmlToText -html  $new_xml $new_html
-  file $new_xml $new_html
-  JIRA_COMMENT_FOOTER="Release Findbugs (version ${findbugs_version}) \
-	warnings: $BUILD_URL/artifact/patchprocess/newFindbugsWarnings.html
-$JIRA_COMMENT_FOOTER"
-  ### if current warnings greater than 0, fail
-  if [[ $findbugsWarnings -gt 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 findbugs{color}.  The patch appears to introduce $findbugsWarnings \
- new Findbugs (version ${findbugs_version}) warnings."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 findbugs{color}.  The patch does not introduce any \
- new Findbugs (version ${findbugs_version}) warnings."
-  return 0
-}
-
-###############################################################################
-### Check line lengths
-checkLineLengths () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking that no line have length > $MAX_LINE_LENGTH"
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  #see http://en.wikipedia.org/wiki/Diff#Unified_format
-
-  MAX_LINE_LENGTH_PATCH=`expr $MAX_LINE_LENGTH + 1`
-  lines=`cat $PATCH_DIR/patch | grep "^+" | grep -v "^@@" | grep -v "^+++" | grep -v "import" | grep -v "org.apache.thrift." | grep -v "com.google.protobuf." | grep -v "protobuf.generated" | awk -v len="$MAX_LINE_LENGTH_PATCH" 'length ($0) > len' | head -n 10`
-  ll=`echo "$lines" | wc -l`
-  if [[ "$ll" -gt "1" ]]; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 lineLengths{color}.  The patch introduces the following lines longer than $MAX_LINE_LENGTH:
-    $lines"
-
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 lineLengths{color}.  The patch does not introduce lines longer than $MAX_LINE_LENGTH"
-  return 0
-}
-
-###############################################################################
-### Run the tests
-runTests () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Running tests."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  failed_tests=""
-  echo "$MVN clean test -Dsurefire.rerunFailingTestsCount=2 -P runAllTests -D${PROJECT_NAME}PatchProcess"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  ulimit -a
-  # Need to export this so the zombie subshell picks up current content
-  export JIRA_COMMENT
-  $MVN clean test -Dsurefire.rerunFailingTestsCount=2 -P runAllTests -D${PROJECT_NAME}PatchProcess
-  if [[ $? != 0 ]] ; then
-     ### Find and format names of failed tests
-     failed_tests=`find . -name 'TEST*.xml' | xargs $GREP  -l -E "<failure|<error" | sed -e "s|.*target/surefire-reports/TEST-|                  |g" | sed -e "s|\.xml||g"`
- 
-     JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 core tests{color}.  The patch failed these unit tests:
-     $failed_tests"
-     JIRA_COMMENT=`$BASEDIR/dev-support/zombie-detector.sh ${BUILD_ID}`
-     return 1
-  else
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 core tests{color}.  The patch passed unit tests in $modules."
-    JIRA_COMMENT=`$BASEDIR/dev-support/zombie-detector.sh ${BUILD_ID}`
-    return $?
-  fi
-}
-
-###############################################################################
-### Check docbook site xml
-checkSiteXml () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking Site generation"
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-
-  echo "$MVN package post-site -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchSiteOutput.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN package post-site -DskipTests -D${PROJECT_NAME}PatchProcess  > $PATCH_DIR/patchSiteOutput.txt 2>&1
-  if [[ $? != 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 site{color}.  The patch appears to cause mvn post-site goal to fail."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 site{color}.  The mvn post-site goal succeeds with this patch."
-  return 0
-}
-
-###############################################################################
-### Run the inject-system-faults target
-checkInjectSystemFaults () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking the integrity of system test framework code."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  
-  ### Kill any rogue build processes from the last attempt
-  $PS auxwww | $GREP ${PROJECT_NAME}PatchProcess | $AWK '{print $2}' | /usr/bin/xargs -t -I {} /bin/kill -9 {} > /dev/null
-
-  #echo "$ANT_HOME/bin/ant -Dversion="${VERSION}" -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME inject-system-faults"
-  #$ANT_HOME/bin/ant -Dversion="${VERSION}" -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME inject-system-faults
-  echo "NOP"
-  return 0
-  if [[ $? != 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 system test framework{color}.  The patch failed system test framework compile."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 system test framework{color}.  The patch passed system test framework compile."
-  return 0
-}
-
-###############################################################################
-### Submit a comment to the defect's Jira
-submitJiraComment () {
-  local result=$1
-  ### Do not output the value of JIRA_COMMENT_FOOTER when run by a developer
-  if [[  $JENKINS == "false" ]] ; then
-    JIRA_COMMENT_FOOTER=""
-  fi
-  if [[ $result == 0 ]] ; then
-    comment="{color:green}+1 overall{color}.  $JIRA_COMMENT
-
-$JIRA_COMMENT_FOOTER"
-  else
-    comment="{color:red}-1 overall{color}.  $JIRA_COMMENT
-
-$JIRA_COMMENT_FOOTER"
-  fi
-  ### Output the test result to the console
-  echo "
-
-
-
-$comment"  
-
-  if [[ $JENKINS == "true" ]] ; then
-    echo ""
-    echo ""
-    echo "======================================================================"
-    echo "======================================================================"
-    echo "    Adding comment to Jira."
-    echo "======================================================================"
-    echo "======================================================================"
-    echo ""
-    echo ""
-    ### Update Jira with a comment
-    export USER=hudson
-    $JIRACLI -s https://issues.apache.org/jira -a addcomment -u hadoopqa -p $JIRA_PASSWD --comment "$comment" --issue $defect
-    $JIRACLI -s https://issues.apache.org/jira -a logout -u hadoopqa -p $JIRA_PASSWD
-  fi
-}
-
-###############################################################################
-### Cleanup files
-cleanupAndExit () {
-  local result=$1
-  if [[ ${JENKINS} == "true" && ${MOVE_PATCH_DIR} == "true" ]] ; then
-    if [ -e "$PATCH_DIR" ] ; then
-      echo "Relocating patch dir into ${BASEDIR}"
-      mv $PATCH_DIR $BASEDIR
-    fi
-  fi
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Finished build."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  exit $result
-}
-
-###############################################################################
-###############################################################################
-###############################################################################
-
-JIRA_COMMENT=""
-JIRA_COMMENT_FOOTER="Console output: $BUILD_URL/console
-
-This message is automatically generated."
-
-### Check if arguments to the script have been specified properly or not
-parseArgs $@
-cd $BASEDIR
-
-echo "Version of this script: Wed Oct 14 00:29:04 PDT 2015"
-checkout
-RESULT=$?
-echo "RESULT = " $RESULT
-if [[ $JENKINS == "true" ]] ; then
-  if [[ $RESULT != 0 ]] ; then
-    exit 100
-  fi
-fi
-setup
-checkAuthor
-RESULT=$?
-echo "RESULT = " $RESULT
-checkTests
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-applyPatch
-if [[ $? != 0 ]] ; then
-  submitJiraComment 1
-  cleanupAndExit 1
-fi
-
-checkAntiPatterns
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkBuildWithHadoopVersions
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkJavacWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkProtocErrors
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkJavadocWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkCheckstyleErrors
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkInterfaceAudience
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkFindbugsWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkReleaseAuditWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkLineLengths
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkSiteXml
-(( RESULT = RESULT + $?))
-echo "RESULT = " $RESULT
-### Do not call these when run by a developer
-if [[ $JENKINS == "true" ]] ; then
-  runTests
-  (( RESULT = RESULT + $? ))
-  echo "RESULT = " $RESULT
-JIRA_COMMENT_FOOTER="Test results: $BUILD_URL/testReport/
-$JIRA_COMMENT_FOOTER"
-fi
-submitJiraComment $RESULT
-cleanupAndExit $RESULT

http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 897b41c..ccc7eb5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1826,6 +1826,9 @@
             <name>HBasePatchProcess</name>
         </property>
       </activation>
+      <properties>
+        <surefire.rerunFailingTestsCount>2</surefire.rerunFailingTestsCount>
+      </properties>
       <build>
         <plugins>
           <plugin>


[43/50] [abbrv] hbase git commit: HBASE-15052 Use EnvironmentEdgeManager in ReplicationSource

Posted by jm...@apache.org.
HBASE-15052 Use EnvironmentEdgeManager in ReplicationSource


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

Branch: refs/heads/trunk
Commit: ec47a811a2e7c0bb3f844d780049bb8a67cc4c35
Parents: 894ce84
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Mon Jan 11 09:37:08 2016 -0800
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Mon Jan 11 09:37:08 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/ec47a811/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 868ddee..b4975bf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
@@ -640,7 +641,7 @@ public class ReplicationSource extends Thread
             sleepMultiplier = 1;
             // if there was nothing to ship and it's not an error
             // set "ageOfLastShippedOp" to <now> to indicate that we're current
-            metrics.setAgeOfLastShippedOp(System.currentTimeMillis(), walGroupId);
+            metrics.setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(), walGroupId);
           }
           if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
             sleepMultiplier++;


[19/50] [abbrv] hbase git commit: HBASE-14888 ClusterSchema: Add Namespace Operations

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
index 177adfd..0cccce1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
@@ -27,16 +27,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
index 078aaa6..6049701 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
@@ -253,5 +253,11 @@ public class TestHFileCleaner {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index d6f1606..0401ae8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -191,5 +191,11 @@ public class TestHFileLinkCleaner {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 f874523..ebf3699 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
@@ -223,5 +223,11 @@ public class TestLogsCleaner {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index 87db386..b13f337 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -260,5 +260,11 @@ public class TestReplicationHFileCleaner {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 77a603d..9731aa4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -438,11 +438,11 @@ public class MasterProcedureTestingUtility {
   }
 
   public static long generateNonceGroup(final HMaster master) {
-    return master.getConnection().getNonceGenerator().getNonceGroup();
+    return master.getClusterConnection().getNonceGenerator().getNonceGroup();
   }
 
   public static long generateNonce(final HMaster master) {
-    return master.getConnection().getNonceGenerator().newNonce();
+    return master.getClusterConnection().getNonceGenerator().newNonce();
   }
 
   public static class InjectAbortOnLoadListener

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index e550c3a..0e38afc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -718,6 +718,12 @@ public class TestHeapMemoryManager {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   static class CustomHeapMemoryTuner implements HeapMemoryTuner {

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
index d62ccde..96ec698 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
@@ -143,6 +143,12 @@ public class TestSplitLogWorker {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   private void waitForCounter(AtomicLong ctr, long oldval, long newval, long timems)

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 3b7402a..94dbb25 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
@@ -191,5 +191,11 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index a082b19..9a878fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -1,4 +1,5 @@
 /**
+
  * 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
@@ -292,5 +293,11 @@ public class TestReplicationTrackerZKImpl {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 a208120..f042a8d 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
@@ -595,5 +595,11 @@ public class TestReplicationSourceManager {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index 69c6e63..faac8eb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -334,6 +334,12 @@ public class TestTokenAuthentication {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   private static HBaseTestingUtility TEST_UTIL;

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
index 1fcfcbb..53e2467 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java
@@ -127,4 +127,10 @@ public class MockServer implements Server {
   public ChoreService getChoreService() {
     return null;
   }
+
+  @Override
+  public ClusterConnection getClusterConnection() {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }
\ No newline at end of file


[21/50] [abbrv] hbase git commit: HBASE-14888 ClusterSchema: Add Namespace Operations

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 6400887..043d549 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -17439,6 +17439,16 @@ public final class MasterProtos {
 
   public interface CreateNamespaceResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional uint64 proc_id = 1;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    boolean hasProcId();
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    long getProcId();
   }
   /**
    * Protobuf type {@code hbase.pb.CreateNamespaceResponse}
@@ -17473,6 +17483,7 @@ public final class MasterProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -17490,6 +17501,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -17529,7 +17545,25 @@ public final class MasterProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional uint64 proc_id = 1;
+    public static final int PROC_ID_FIELD_NUMBER = 1;
+    private long procId_;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public boolean hasProcId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public long getProcId() {
+      return procId_;
+    }
+
     private void initFields() {
+      procId_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -17543,6 +17577,9 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, procId_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -17552,6 +17589,10 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, procId_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -17575,6 +17616,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -17588,6 +17634,10 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasProcId()) {
+        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getProcId());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -17697,6 +17747,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -17723,6 +17775,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.procId_ = procId_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -17738,6 +17797,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -17763,6 +17825,40 @@ public final class MasterProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional uint64 proc_id = 1;
+      private long procId_ ;
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000001;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:hbase.pb.CreateNamespaceResponse)
     }
@@ -18474,6 +18570,16 @@ public final class MasterProtos {
 
   public interface DeleteNamespaceResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional uint64 proc_id = 1;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    boolean hasProcId();
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    long getProcId();
   }
   /**
    * Protobuf type {@code hbase.pb.DeleteNamespaceResponse}
@@ -18508,6 +18614,7 @@ public final class MasterProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -18525,6 +18632,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -18564,7 +18676,25 @@ public final class MasterProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional uint64 proc_id = 1;
+    public static final int PROC_ID_FIELD_NUMBER = 1;
+    private long procId_;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public boolean hasProcId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public long getProcId() {
+      return procId_;
+    }
+
     private void initFields() {
+      procId_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -18578,6 +18708,9 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, procId_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -18587,6 +18720,10 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, procId_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -18610,6 +18747,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -18623,6 +18765,10 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasProcId()) {
+        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getProcId());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -18732,6 +18878,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -18758,6 +18906,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.procId_ = procId_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -18773,6 +18928,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -18798,6 +18956,40 @@ public final class MasterProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional uint64 proc_id = 1;
+      private long procId_ ;
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000001;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:hbase.pb.DeleteNamespaceResponse)
     }
@@ -19553,6 +19745,16 @@ public final class MasterProtos {
 
   public interface ModifyNamespaceResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional uint64 proc_id = 1;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    boolean hasProcId();
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    long getProcId();
   }
   /**
    * Protobuf type {@code hbase.pb.ModifyNamespaceResponse}
@@ -19587,6 +19789,7 @@ public final class MasterProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -19604,6 +19807,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -19643,7 +19851,25 @@ public final class MasterProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional uint64 proc_id = 1;
+    public static final int PROC_ID_FIELD_NUMBER = 1;
+    private long procId_;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public boolean hasProcId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public long getProcId() {
+      return procId_;
+    }
+
     private void initFields() {
+      procId_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -19657,6 +19883,9 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, procId_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -19666,6 +19895,10 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, procId_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -19689,6 +19922,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -19702,6 +19940,10 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasProcId()) {
+        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getProcId());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -19811,6 +20053,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -19837,6 +20081,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.procId_ = procId_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -19852,6 +20103,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -19877,6 +20131,40 @@ public final class MasterProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional uint64 proc_id = 1;
+      private long procId_ ;
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000001;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:hbase.pb.ModifyNamespaceResponse)
     }
@@ -61891,255 +62179,257 @@ public final class MasterProtos {
       "\001(\004\"~\n\026CreateNamespaceRequest\022:\n\023namespa" +
       "ceDescriptor\030\001 \002(\0132\035.hbase.pb.NamespaceD" +
       "escriptor\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005non",
-      "ce\030\003 \001(\004:\0010\"\031\n\027CreateNamespaceResponse\"Y" +
-      "\n\026DeleteNamespaceRequest\022\025\n\rnamespaceNam" +
-      "e\030\001 \002(\t\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce" +
-      "\030\003 \001(\004:\0010\"\031\n\027DeleteNamespaceResponse\"~\n\026" +
-      "ModifyNamespaceRequest\022:\n\023namespaceDescr" +
-      "iptor\030\001 \002(\0132\035.hbase.pb.NamespaceDescript" +
-      "or\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(" +
-      "\004:\0010\"\031\n\027ModifyNamespaceResponse\"6\n\035GetNa" +
-      "mespaceDescriptorRequest\022\025\n\rnamespaceNam" +
-      "e\030\001 \002(\t\"\\\n\036GetNamespaceDescriptorRespons",
-      "e\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.hbase.p" +
-      "b.NamespaceDescriptor\"!\n\037ListNamespaceDe" +
-      "scriptorsRequest\"^\n ListNamespaceDescrip" +
-      "torsResponse\022:\n\023namespaceDescriptor\030\001 \003(" +
-      "\0132\035.hbase.pb.NamespaceDescriptor\"?\n&List" +
-      "TableDescriptorsByNamespaceRequest\022\025\n\rna" +
-      "mespaceName\030\001 \002(\t\"U\n\'ListTableDescriptor" +
-      "sByNamespaceResponse\022*\n\013tableSchema\030\001 \003(" +
-      "\0132\025.hbase.pb.TableSchema\"9\n ListTableNam" +
-      "esByNamespaceRequest\022\025\n\rnamespaceName\030\001 ",
-      "\002(\t\"K\n!ListTableNamesByNamespaceResponse" +
-      "\022&\n\ttableName\030\001 \003(\0132\023.hbase.pb.TableName" +
-      "\"\021\n\017ShutdownRequest\"\022\n\020ShutdownResponse\"" +
-      "\023\n\021StopMasterRequest\"\024\n\022StopMasterRespon" +
-      "se\"\037\n\016BalanceRequest\022\r\n\005force\030\001 \001(\010\"\'\n\017B" +
-      "alanceResponse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031" +
-      "SetBalancerRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n" +
-      "\013synchronous\030\002 \001(\010\"8\n\032SetBalancerRunning" +
-      "Response\022\032\n\022prev_balance_value\030\001 \001(\010\"\032\n\030" +
-      "IsBalancerEnabledRequest\",\n\031IsBalancerEn",
-      "abledResponse\022\017\n\007enabled\030\001 \002(\010\"\022\n\020Normal" +
-      "izeRequest\"+\n\021NormalizeResponse\022\026\n\016norma" +
-      "lizer_ran\030\001 \002(\010\")\n\033SetNormalizerRunningR" +
-      "equest\022\n\n\002on\030\001 \002(\010\"=\n\034SetNormalizerRunni" +
-      "ngResponse\022\035\n\025prev_normalizer_value\030\001 \001(" +
-      "\010\"\034\n\032IsNormalizerEnabledRequest\".\n\033IsNor" +
-      "malizerEnabledResponse\022\017\n\007enabled\030\001 \002(\010\"" +
-      "\027\n\025RunCatalogScanRequest\"-\n\026RunCatalogSc" +
-      "anResponse\022\023\n\013scan_result\030\001 \001(\005\"-\n\033Enabl" +
-      "eCatalogJanitorRequest\022\016\n\006enable\030\001 \002(\010\"2",
-      "\n\034EnableCatalogJanitorResponse\022\022\n\nprev_v" +
-      "alue\030\001 \001(\010\" \n\036IsCatalogJanitorEnabledReq" +
-      "uest\"0\n\037IsCatalogJanitorEnabledResponse\022" +
-      "\r\n\005value\030\001 \002(\010\"B\n\017SnapshotRequest\022/\n\010sna" +
-      "pshot\030\001 \002(\0132\035.hbase.pb.SnapshotDescripti" +
-      "on\",\n\020SnapshotResponse\022\030\n\020expected_timeo" +
-      "ut\030\001 \002(\003\"\036\n\034GetCompletedSnapshotsRequest" +
-      "\"Q\n\035GetCompletedSnapshotsResponse\0220\n\tsna" +
-      "pshots\030\001 \003(\0132\035.hbase.pb.SnapshotDescript" +
-      "ion\"H\n\025DeleteSnapshotRequest\022/\n\010snapshot",
-      "\030\001 \002(\0132\035.hbase.pb.SnapshotDescription\"\030\n" +
-      "\026DeleteSnapshotResponse\"I\n\026RestoreSnapsh" +
-      "otRequest\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.S" +
-      "napshotDescription\"\031\n\027RestoreSnapshotRes" +
-      "ponse\"H\n\025IsSnapshotDoneRequest\022/\n\010snapsh" +
-      "ot\030\001 \001(\0132\035.hbase.pb.SnapshotDescription\"" +
-      "^\n\026IsSnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:" +
-      "\005false\022/\n\010snapshot\030\002 \001(\0132\035.hbase.pb.Snap" +
-      "shotDescription\"O\n\034IsRestoreSnapshotDone" +
-      "Request\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.Sna",
-      "pshotDescription\"4\n\035IsRestoreSnapshotDon" +
-      "eResponse\022\023\n\004done\030\001 \001(\010:\005false\"F\n\033GetSch" +
-      "emaAlterStatusRequest\022\'\n\ntable_name\030\001 \002(" +
-      "\0132\023.hbase.pb.TableName\"T\n\034GetSchemaAlter" +
-      "StatusResponse\022\035\n\025yet_to_update_regions\030" +
-      "\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"\213\001\n\032GetTabl" +
-      "eDescriptorsRequest\022(\n\013table_names\030\001 \003(\013" +
-      "2\023.hbase.pb.TableName\022\r\n\005regex\030\002 \001(\t\022!\n\022" +
-      "include_sys_tables\030\003 \001(\010:\005false\022\021\n\tnames" +
-      "pace\030\004 \001(\t\"J\n\033GetTableDescriptorsRespons",
-      "e\022+\n\014table_schema\030\001 \003(\0132\025.hbase.pb.Table" +
-      "Schema\"[\n\024GetTableNamesRequest\022\r\n\005regex\030" +
-      "\001 \001(\t\022!\n\022include_sys_tables\030\002 \001(\010:\005false" +
-      "\022\021\n\tnamespace\030\003 \001(\t\"A\n\025GetTableNamesResp" +
-      "onse\022(\n\013table_names\030\001 \003(\0132\023.hbase.pb.Tab" +
-      "leName\"?\n\024GetTableStateRequest\022\'\n\ntable_" +
-      "name\030\001 \002(\0132\023.hbase.pb.TableName\"B\n\025GetTa" +
-      "bleStateResponse\022)\n\013table_state\030\001 \002(\0132\024." +
-      "hbase.pb.TableState\"\031\n\027GetClusterStatusR" +
-      "equest\"K\n\030GetClusterStatusResponse\022/\n\016cl",
-      "uster_status\030\001 \002(\0132\027.hbase.pb.ClusterSta" +
-      "tus\"\030\n\026IsMasterRunningRequest\"4\n\027IsMaste" +
-      "rRunningResponse\022\031\n\021is_master_running\030\001 " +
-      "\002(\010\"I\n\024ExecProcedureRequest\0221\n\tprocedure" +
-      "\030\001 \002(\0132\036.hbase.pb.ProcedureDescription\"F" +
-      "\n\025ExecProcedureResponse\022\030\n\020expected_time" +
-      "out\030\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"K\n\026IsProc" +
-      "edureDoneRequest\0221\n\tprocedure\030\001 \001(\0132\036.hb" +
-      "ase.pb.ProcedureDescription\"`\n\027IsProcedu" +
-      "reDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\0220\n\010s",
-      "napshot\030\002 \001(\0132\036.hbase.pb.ProcedureDescri" +
-      "ption\",\n\031GetProcedureResultRequest\022\017\n\007pr" +
-      "oc_id\030\001 \002(\004\"\371\001\n\032GetProcedureResultRespon" +
-      "se\0229\n\005state\030\001 \002(\0162*.hbase.pb.GetProcedur" +
-      "eResultResponse.State\022\022\n\nstart_time\030\002 \001(" +
-      "\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224" +
-      "\n\texception\030\005 \001(\0132!.hbase.pb.ForeignExce" +
-      "ptionMessage\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007" +
-      "RUNNING\020\001\022\014\n\010FINISHED\020\002\"M\n\025AbortProcedur" +
-      "eRequest\022\017\n\007proc_id\030\001 \002(\004\022#\n\025mayInterrup",
-      "tIfRunning\030\002 \001(\010:\004true\"6\n\026AbortProcedure" +
-      "Response\022\034\n\024is_procedure_aborted\030\001 \002(\010\"\027" +
-      "\n\025ListProceduresRequest\"@\n\026ListProcedure" +
-      "sResponse\022&\n\tprocedure\030\001 \003(\0132\023.hbase.pb." +
-      "Procedure\"\315\001\n\017SetQuotaRequest\022\021\n\tuser_na" +
-      "me\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tnamespac" +
-      "e\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hbase.pb.T" +
-      "ableName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypass_g" +
-      "lobals\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031.hbase.p" +
-      "b.ThrottleRequest\"\022\n\020SetQuotaResponse\"J\n",
-      "\037MajorCompactionTimestampRequest\022\'\n\ntabl" +
-      "e_name\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(Maj" +
-      "orCompactionTimestampForRegionRequest\022)\n" +
-      "\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifier" +
-      "\"@\n MajorCompactionTimestampResponse\022\034\n\024" +
-      "compaction_timestamp\030\001 \002(\003\"\035\n\033SecurityCa" +
-      "pabilitiesRequest\"\354\001\n\034SecurityCapabiliti" +
-      "esResponse\022G\n\014capabilities\030\001 \003(\01621.hbase" +
-      ".pb.SecurityCapabilitiesResponse.Capabil" +
-      "ity\"\202\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTICAT",
-      "ION\020\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rAUTH" +
-      "ORIZATION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017C" +
-      "ELL_VISIBILITY\020\0042\374&\n\rMasterService\022e\n\024Ge" +
-      "tSchemaAlterStatus\022%.hbase.pb.GetSchemaA" +
-      "lterStatusRequest\032&.hbase.pb.GetSchemaAl" +
-      "terStatusResponse\022b\n\023GetTableDescriptors" +
-      "\022$.hbase.pb.GetTableDescriptorsRequest\032%" +
-      ".hbase.pb.GetTableDescriptorsResponse\022P\n" +
-      "\rGetTableNames\022\036.hbase.pb.GetTableNamesR" +
-      "equest\032\037.hbase.pb.GetTableNamesResponse\022",
-      "Y\n\020GetClusterStatus\022!.hbase.pb.GetCluste" +
-      "rStatusRequest\032\".hbase.pb.GetClusterStat" +
-      "usResponse\022V\n\017IsMasterRunning\022 .hbase.pb" +
-      ".IsMasterRunningRequest\032!.hbase.pb.IsMas" +
-      "terRunningResponse\022D\n\tAddColumn\022\032.hbase." +
-      "pb.AddColumnRequest\032\033.hbase.pb.AddColumn" +
-      "Response\022M\n\014DeleteColumn\022\035.hbase.pb.Dele" +
-      "teColumnRequest\032\036.hbase.pb.DeleteColumnR" +
-      "esponse\022M\n\014ModifyColumn\022\035.hbase.pb.Modif" +
-      "yColumnRequest\032\036.hbase.pb.ModifyColumnRe",
-      "sponse\022G\n\nMoveRegion\022\033.hbase.pb.MoveRegi" +
-      "onRequest\032\034.hbase.pb.MoveRegionResponse\022" +
-      "k\n\026DispatchMergingRegions\022\'.hbase.pb.Dis" +
-      "patchMergingRegionsRequest\032(.hbase.pb.Di" +
-      "spatchMergingRegionsResponse\022M\n\014AssignRe" +
-      "gion\022\035.hbase.pb.AssignRegionRequest\032\036.hb" +
-      "ase.pb.AssignRegionResponse\022S\n\016UnassignR" +
-      "egion\022\037.hbase.pb.UnassignRegionRequest\032 " +
-      ".hbase.pb.UnassignRegionResponse\022P\n\rOffl" +
-      "ineRegion\022\036.hbase.pb.OfflineRegionReques",
-      "t\032\037.hbase.pb.OfflineRegionResponse\022J\n\013De" +
-      "leteTable\022\034.hbase.pb.DeleteTableRequest\032" +
-      "\035.hbase.pb.DeleteTableResponse\022P\n\rtrunca" +
-      "teTable\022\036.hbase.pb.TruncateTableRequest\032" +
-      "\037.hbase.pb.TruncateTableResponse\022J\n\013Enab" +
-      "leTable\022\034.hbase.pb.EnableTableRequest\032\035." +
-      "hbase.pb.EnableTableResponse\022M\n\014DisableT" +
-      "able\022\035.hbase.pb.DisableTableRequest\032\036.hb" +
-      "ase.pb.DisableTableResponse\022J\n\013ModifyTab" +
-      "le\022\034.hbase.pb.ModifyTableRequest\032\035.hbase",
-      ".pb.ModifyTableResponse\022J\n\013CreateTable\022\034" +
-      ".hbase.pb.CreateTableRequest\032\035.hbase.pb." +
-      "CreateTableResponse\022A\n\010Shutdown\022\031.hbase." +
-      "pb.ShutdownRequest\032\032.hbase.pb.ShutdownRe" +
-      "sponse\022G\n\nStopMaster\022\033.hbase.pb.StopMast" +
-      "erRequest\032\034.hbase.pb.StopMasterResponse\022" +
-      ">\n\007Balance\022\030.hbase.pb.BalanceRequest\032\031.h" +
-      "base.pb.BalanceResponse\022_\n\022SetBalancerRu" +
-      "nning\022#.hbase.pb.SetBalancerRunningReque" +
-      "st\032$.hbase.pb.SetBalancerRunningResponse",
-      "\022\\\n\021IsBalancerEnabled\022\".hbase.pb.IsBalan" +
-      "cerEnabledRequest\032#.hbase.pb.IsBalancerE" +
-      "nabledResponse\022D\n\tNormalize\022\032.hbase.pb.N" +
-      "ormalizeRequest\032\033.hbase.pb.NormalizeResp" +
-      "onse\022e\n\024SetNormalizerRunning\022%.hbase.pb." +
-      "SetNormalizerRunningRequest\032&.hbase.pb.S" +
-      "etNormalizerRunningResponse\022b\n\023IsNormali" +
-      "zerEnabled\022$.hbase.pb.IsNormalizerEnable" +
-      "dRequest\032%.hbase.pb.IsNormalizerEnabledR" +
-      "esponse\022S\n\016RunCatalogScan\022\037.hbase.pb.Run",
-      "CatalogScanRequest\032 .hbase.pb.RunCatalog" +
-      "ScanResponse\022e\n\024EnableCatalogJanitor\022%.h" +
-      "base.pb.EnableCatalogJanitorRequest\032&.hb" +
-      "ase.pb.EnableCatalogJanitorResponse\022n\n\027I" +
-      "sCatalogJanitorEnabled\022(.hbase.pb.IsCata" +
-      "logJanitorEnabledRequest\032).hbase.pb.IsCa" +
-      "talogJanitorEnabledResponse\022^\n\021ExecMaste" +
-      "rService\022#.hbase.pb.CoprocessorServiceRe" +
-      "quest\032$.hbase.pb.CoprocessorServiceRespo" +
-      "nse\022A\n\010Snapshot\022\031.hbase.pb.SnapshotReque",
-      "st\032\032.hbase.pb.SnapshotResponse\022h\n\025GetCom" +
-      "pletedSnapshots\022&.hbase.pb.GetCompletedS" +
-      "napshotsRequest\032\'.hbase.pb.GetCompletedS" +
-      "napshotsResponse\022S\n\016DeleteSnapshot\022\037.hba" +
-      "se.pb.DeleteSnapshotRequest\032 .hbase.pb.D" +
-      "eleteSnapshotResponse\022S\n\016IsSnapshotDone\022" +
-      "\037.hbase.pb.IsSnapshotDoneRequest\032 .hbase" +
-      ".pb.IsSnapshotDoneResponse\022V\n\017RestoreSna" +
-      "pshot\022 .hbase.pb.RestoreSnapshotRequest\032" +
-      "!.hbase.pb.RestoreSnapshotResponse\022h\n\025Is",
-      "RestoreSnapshotDone\022&.hbase.pb.IsRestore" +
-      "SnapshotDoneRequest\032\'.hbase.pb.IsRestore" +
-      "SnapshotDoneResponse\022P\n\rExecProcedure\022\036." +
-      "hbase.pb.ExecProcedureRequest\032\037.hbase.pb" +
-      ".ExecProcedureResponse\022W\n\024ExecProcedureW" +
-      "ithRet\022\036.hbase.pb.ExecProcedureRequest\032\037" +
-      ".hbase.pb.ExecProcedureResponse\022V\n\017IsPro" +
-      "cedureDone\022 .hbase.pb.IsProcedureDoneReq" +
-      "uest\032!.hbase.pb.IsProcedureDoneResponse\022" +
-      "V\n\017ModifyNamespace\022 .hbase.pb.ModifyName",
-      "spaceRequest\032!.hbase.pb.ModifyNamespaceR" +
-      "esponse\022V\n\017CreateNamespace\022 .hbase.pb.Cr" +
-      "eateNamespaceRequest\032!.hbase.pb.CreateNa" +
-      "mespaceResponse\022V\n\017DeleteNamespace\022 .hba" +
-      "se.pb.DeleteNamespaceRequest\032!.hbase.pb." +
-      "DeleteNamespaceResponse\022k\n\026GetNamespaceD" +
-      "escriptor\022\'.hbase.pb.GetNamespaceDescrip" +
-      "torRequest\032(.hbase.pb.GetNamespaceDescri" +
-      "ptorResponse\022q\n\030ListNamespaceDescriptors" +
-      "\022).hbase.pb.ListNamespaceDescriptorsRequ",
-      "est\032*.hbase.pb.ListNamespaceDescriptorsR" +
-      "esponse\022\206\001\n\037ListTableDescriptorsByNamesp" +
-      "ace\0220.hbase.pb.ListTableDescriptorsByNam" +
-      "espaceRequest\0321.hbase.pb.ListTableDescri" +
-      "ptorsByNamespaceResponse\022t\n\031ListTableNam" +
-      "esByNamespace\022*.hbase.pb.ListTableNamesB" +
-      "yNamespaceRequest\032+.hbase.pb.ListTableNa" +
-      "mesByNamespaceResponse\022P\n\rGetTableState\022" +
-      "\036.hbase.pb.GetTableStateRequest\032\037.hbase." +
-      "pb.GetTableStateResponse\022A\n\010SetQuota\022\031.h",
-      "base.pb.SetQuotaRequest\032\032.hbase.pb.SetQu" +
-      "otaResponse\022x\n\037getLastMajorCompactionTim" +
-      "estamp\022).hbase.pb.MajorCompactionTimesta" +
-      "mpRequest\032*.hbase.pb.MajorCompactionTime" +
-      "stampResponse\022\212\001\n(getLastMajorCompaction" +
-      "TimestampForRegion\0222.hbase.pb.MajorCompa" +
-      "ctionTimestampForRegionRequest\032*.hbase.p" +
-      "b.MajorCompactionTimestampResponse\022_\n\022ge" +
-      "tProcedureResult\022#.hbase.pb.GetProcedure" +
-      "ResultRequest\032$.hbase.pb.GetProcedureRes",
-      "ultResponse\022h\n\027getSecurityCapabilities\022%" +
-      ".hbase.pb.SecurityCapabilitiesRequest\032&." +
-      "hbase.pb.SecurityCapabilitiesResponse\022S\n" +
-      "\016AbortProcedure\022\037.hbase.pb.AbortProcedur" +
-      "eRequest\032 .hbase.pb.AbortProcedureRespon" +
-      "se\022S\n\016ListProcedures\022\037.hbase.pb.ListProc" +
-      "eduresRequest\032 .hbase.pb.ListProceduresR" +
-      "esponseBB\n*org.apache.hadoop.hbase.proto" +
-      "buf.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
+      "ce\030\003 \001(\004:\0010\"*\n\027CreateNamespaceResponse\022\017" +
+      "\n\007proc_id\030\001 \001(\004\"Y\n\026DeleteNamespaceReques" +
+      "t\022\025\n\rnamespaceName\030\001 \002(\t\022\026\n\013nonce_group\030" +
+      "\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027DeleteName" +
+      "spaceResponse\022\017\n\007proc_id\030\001 \001(\004\"~\n\026Modify" +
+      "NamespaceRequest\022:\n\023namespaceDescriptor\030" +
+      "\001 \002(\0132\035.hbase.pb.NamespaceDescriptor\022\026\n\013" +
+      "nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*" +
+      "\n\027ModifyNamespaceResponse\022\017\n\007proc_id\030\001 \001" +
+      "(\004\"6\n\035GetNamespaceDescriptorRequest\022\025\n\rn",
+      "amespaceName\030\001 \002(\t\"\\\n\036GetNamespaceDescri" +
+      "ptorResponse\022:\n\023namespaceDescriptor\030\001 \002(" +
+      "\0132\035.hbase.pb.NamespaceDescriptor\"!\n\037List" +
+      "NamespaceDescriptorsRequest\"^\n ListNames" +
+      "paceDescriptorsResponse\022:\n\023namespaceDesc" +
+      "riptor\030\001 \003(\0132\035.hbase.pb.NamespaceDescrip" +
+      "tor\"?\n&ListTableDescriptorsByNamespaceRe" +
+      "quest\022\025\n\rnamespaceName\030\001 \002(\t\"U\n\'ListTabl" +
+      "eDescriptorsByNamespaceResponse\022*\n\013table" +
+      "Schema\030\001 \003(\0132\025.hbase.pb.TableSchema\"9\n L",
+      "istTableNamesByNamespaceRequest\022\025\n\rnames" +
+      "paceName\030\001 \002(\t\"K\n!ListTableNamesByNamesp" +
+      "aceResponse\022&\n\ttableName\030\001 \003(\0132\023.hbase.p" +
+      "b.TableName\"\021\n\017ShutdownRequest\"\022\n\020Shutdo" +
+      "wnResponse\"\023\n\021StopMasterRequest\"\024\n\022StopM" +
+      "asterResponse\"\037\n\016BalanceRequest\022\r\n\005force" +
+      "\030\001 \001(\010\"\'\n\017BalanceResponse\022\024\n\014balancer_ra" +
+      "n\030\001 \002(\010\"<\n\031SetBalancerRunningRequest\022\n\n\002" +
+      "on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8\n\032SetBala" +
+      "ncerRunningResponse\022\032\n\022prev_balance_valu",
+      "e\030\001 \001(\010\"\032\n\030IsBalancerEnabledRequest\",\n\031I" +
+      "sBalancerEnabledResponse\022\017\n\007enabled\030\001 \002(" +
+      "\010\"\022\n\020NormalizeRequest\"+\n\021NormalizeRespon" +
+      "se\022\026\n\016normalizer_ran\030\001 \002(\010\")\n\033SetNormali" +
+      "zerRunningRequest\022\n\n\002on\030\001 \002(\010\"=\n\034SetNorm" +
+      "alizerRunningResponse\022\035\n\025prev_normalizer" +
+      "_value\030\001 \001(\010\"\034\n\032IsNormalizerEnabledReque" +
+      "st\".\n\033IsNormalizerEnabledResponse\022\017\n\007ena" +
+      "bled\030\001 \002(\010\"\027\n\025RunCatalogScanRequest\"-\n\026R" +
+      "unCatalogScanResponse\022\023\n\013scan_result\030\001 \001",
+      "(\005\"-\n\033EnableCatalogJanitorRequest\022\016\n\006ena" +
+      "ble\030\001 \002(\010\"2\n\034EnableCatalogJanitorRespons" +
+      "e\022\022\n\nprev_value\030\001 \001(\010\" \n\036IsCatalogJanito" +
+      "rEnabledRequest\"0\n\037IsCatalogJanitorEnabl" +
+      "edResponse\022\r\n\005value\030\001 \002(\010\"B\n\017SnapshotReq" +
+      "uest\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.Snapsh" +
+      "otDescription\",\n\020SnapshotResponse\022\030\n\020exp" +
+      "ected_timeout\030\001 \002(\003\"\036\n\034GetCompletedSnaps" +
+      "hotsRequest\"Q\n\035GetCompletedSnapshotsResp" +
+      "onse\0220\n\tsnapshots\030\001 \003(\0132\035.hbase.pb.Snaps",
+      "hotDescription\"H\n\025DeleteSnapshotRequest\022" +
+      "/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.SnapshotDes" +
+      "cription\"\030\n\026DeleteSnapshotResponse\"I\n\026Re" +
+      "storeSnapshotRequest\022/\n\010snapshot\030\001 \002(\0132\035" +
+      ".hbase.pb.SnapshotDescription\"\031\n\027Restore" +
+      "SnapshotResponse\"H\n\025IsSnapshotDoneReques" +
+      "t\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.SnapshotD" +
+      "escription\"^\n\026IsSnapshotDoneResponse\022\023\n\004" +
+      "done\030\001 \001(\010:\005false\022/\n\010snapshot\030\002 \001(\0132\035.hb" +
+      "ase.pb.SnapshotDescription\"O\n\034IsRestoreS",
+      "napshotDoneRequest\022/\n\010snapshot\030\001 \001(\0132\035.h" +
+      "base.pb.SnapshotDescription\"4\n\035IsRestore" +
+      "SnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:\005fals" +
+      "e\"F\n\033GetSchemaAlterStatusRequest\022\'\n\ntabl" +
+      "e_name\030\001 \002(\0132\023.hbase.pb.TableName\"T\n\034Get" +
+      "SchemaAlterStatusResponse\022\035\n\025yet_to_upda" +
+      "te_regions\030\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"" +
+      "\213\001\n\032GetTableDescriptorsRequest\022(\n\013table_" +
+      "names\030\001 \003(\0132\023.hbase.pb.TableName\022\r\n\005rege" +
+      "x\030\002 \001(\t\022!\n\022include_sys_tables\030\003 \001(\010:\005fal",
+      "se\022\021\n\tnamespace\030\004 \001(\t\"J\n\033GetTableDescrip" +
+      "torsResponse\022+\n\014table_schema\030\001 \003(\0132\025.hba" +
+      "se.pb.TableSchema\"[\n\024GetTableNamesReques" +
+      "t\022\r\n\005regex\030\001 \001(\t\022!\n\022include_sys_tables\030\002" +
+      " \001(\010:\005false\022\021\n\tnamespace\030\003 \001(\t\"A\n\025GetTab" +
+      "leNamesResponse\022(\n\013table_names\030\001 \003(\0132\023.h" +
+      "base.pb.TableName\"?\n\024GetTableStateReques" +
+      "t\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableNa" +
+      "me\"B\n\025GetTableStateResponse\022)\n\013table_sta" +
+      "te\030\001 \002(\0132\024.hbase.pb.TableState\"\031\n\027GetClu",
+      "sterStatusRequest\"K\n\030GetClusterStatusRes" +
+      "ponse\022/\n\016cluster_status\030\001 \002(\0132\027.hbase.pb" +
+      ".ClusterStatus\"\030\n\026IsMasterRunningRequest" +
+      "\"4\n\027IsMasterRunningResponse\022\031\n\021is_master" +
+      "_running\030\001 \002(\010\"I\n\024ExecProcedureRequest\0221" +
+      "\n\tprocedure\030\001 \002(\0132\036.hbase.pb.ProcedureDe" +
+      "scription\"F\n\025ExecProcedureResponse\022\030\n\020ex" +
+      "pected_timeout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(" +
+      "\014\"K\n\026IsProcedureDoneRequest\0221\n\tprocedure" +
+      "\030\001 \001(\0132\036.hbase.pb.ProcedureDescription\"`",
+      "\n\027IsProcedureDoneResponse\022\023\n\004done\030\001 \001(\010:" +
+      "\005false\0220\n\010snapshot\030\002 \001(\0132\036.hbase.pb.Proc" +
+      "edureDescription\",\n\031GetProcedureResultRe" +
+      "quest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032GetProcedureR" +
+      "esultResponse\0229\n\005state\030\001 \002(\0162*.hbase.pb." +
+      "GetProcedureResultResponse.State\022\022\n\nstar" +
+      "t_time\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006res" +
+      "ult\030\004 \001(\014\0224\n\texception\030\005 \001(\0132!.hbase.pb." +
+      "ForeignExceptionMessage\"1\n\005State\022\r\n\tNOT_" +
+      "FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"M\n\025Ab",
+      "ortProcedureRequest\022\017\n\007proc_id\030\001 \002(\004\022#\n\025" +
+      "mayInterruptIfRunning\030\002 \001(\010:\004true\"6\n\026Abo" +
+      "rtProcedureResponse\022\034\n\024is_procedure_abor" +
+      "ted\030\001 \002(\010\"\027\n\025ListProceduresRequest\"@\n\026Li" +
+      "stProceduresResponse\022&\n\tprocedure\030\001 \003(\0132" +
+      "\023.hbase.pb.Procedure\"\315\001\n\017SetQuotaRequest" +
+      "\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022" +
+      "\021\n\tnamespace\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023" +
+      ".hbase.pb.TableName\022\022\n\nremove_all\030\005 \001(\010\022" +
+      "\026\n\016bypass_globals\030\006 \001(\010\022+\n\010throttle\030\007 \001(",
+      "\0132\031.hbase.pb.ThrottleRequest\"\022\n\020SetQuota" +
+      "Response\"J\n\037MajorCompactionTimestampRequ" +
+      "est\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.Table" +
+      "Name\"U\n(MajorCompactionTimestampForRegio" +
+      "nRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Regi" +
+      "onSpecifier\"@\n MajorCompactionTimestampR" +
+      "esponse\022\034\n\024compaction_timestamp\030\001 \002(\003\"\035\n" +
+      "\033SecurityCapabilitiesRequest\"\354\001\n\034Securit" +
+      "yCapabilitiesResponse\022G\n\014capabilities\030\001 " +
+      "\003(\01621.hbase.pb.SecurityCapabilitiesRespo",
+      "nse.Capability\"\202\001\n\nCapability\022\031\n\025SIMPLE_" +
+      "AUTHENTICATION\020\000\022\031\n\025SECURE_AUTHENTICATIO" +
+      "N\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022CELL_AUTHORIZA" +
+      "TION\020\003\022\023\n\017CELL_VISIBILITY\020\0042\374&\n\rMasterSe" +
+      "rvice\022e\n\024GetSchemaAlterStatus\022%.hbase.pb" +
+      ".GetSchemaAlterStatusRequest\032&.hbase.pb." +
+      "GetSchemaAlterStatusResponse\022b\n\023GetTable" +
+      "Descriptors\022$.hbase.pb.GetTableDescripto" +
+      "rsRequest\032%.hbase.pb.GetTableDescriptors" +
+      "Response\022P\n\rGetTableNames\022\036.hbase.pb.Get",
+      "TableNamesRequest\032\037.hbase.pb.GetTableNam" +
+      "esResponse\022Y\n\020GetClusterStatus\022!.hbase.p" +
+      "b.GetClusterStatusRequest\032\".hbase.pb.Get" +
+      "ClusterStatusResponse\022V\n\017IsMasterRunning" +
+      "\022 .hbase.pb.IsMasterRunningRequest\032!.hba" +
+      "se.pb.IsMasterRunningResponse\022D\n\tAddColu" +
+      "mn\022\032.hbase.pb.AddColumnRequest\032\033.hbase.p" +
+      "b.AddColumnResponse\022M\n\014DeleteColumn\022\035.hb" +
+      "ase.pb.DeleteColumnRequest\032\036.hbase.pb.De" +
+      "leteColumnResponse\022M\n\014ModifyColumn\022\035.hba",
+      "se.pb.ModifyColumnRequest\032\036.hbase.pb.Mod" +
+      "ifyColumnResponse\022G\n\nMoveRegion\022\033.hbase." +
+      "pb.MoveRegionRequest\032\034.hbase.pb.MoveRegi" +
+      "onResponse\022k\n\026DispatchMergingRegions\022\'.h" +
+      "base.pb.DispatchMergingRegionsRequest\032(." +
+      "hbase.pb.DispatchMergingRegionsResponse\022" +
+      "M\n\014AssignRegion\022\035.hbase.pb.AssignRegionR" +
+      "equest\032\036.hbase.pb.AssignRegionResponse\022S" +
+      "\n\016UnassignRegion\022\037.hbase.pb.UnassignRegi" +
+      "onRequest\032 .hbase.pb.UnassignRegionRespo",
+      "nse\022P\n\rOfflineRegion\022\036.hbase.pb.OfflineR" +
+      "egionRequest\032\037.hbase.pb.OfflineRegionRes" +
+      "ponse\022J\n\013DeleteTable\022\034.hbase.pb.DeleteTa" +
+      "bleRequest\032\035.hbase.pb.DeleteTableRespons" +
+      "e\022P\n\rtruncateTable\022\036.hbase.pb.TruncateTa" +
+      "bleRequest\032\037.hbase.pb.TruncateTableRespo" +
+      "nse\022J\n\013EnableTable\022\034.hbase.pb.EnableTabl" +
+      "eRequest\032\035.hbase.pb.EnableTableResponse\022" +
+      "M\n\014DisableTable\022\035.hbase.pb.DisableTableR" +
+      "equest\032\036.hbase.pb.DisableTableResponse\022J",
+      "\n\013ModifyTable\022\034.hbase.pb.ModifyTableRequ" +
+      "est\032\035.hbase.pb.ModifyTableResponse\022J\n\013Cr" +
+      "eateTable\022\034.hbase.pb.CreateTableRequest\032" +
+      "\035.hbase.pb.CreateTableResponse\022A\n\010Shutdo" +
+      "wn\022\031.hbase.pb.ShutdownRequest\032\032.hbase.pb" +
+      ".ShutdownResponse\022G\n\nStopMaster\022\033.hbase." +
+      "pb.StopMasterRequest\032\034.hbase.pb.StopMast" +
+      "erResponse\022>\n\007Balance\022\030.hbase.pb.Balance" +
+      "Request\032\031.hbase.pb.BalanceResponse\022_\n\022Se" +
+      "tBalancerRunning\022#.hbase.pb.SetBalancerR",
+      "unningRequest\032$.hbase.pb.SetBalancerRunn" +
+      "ingResponse\022\\\n\021IsBalancerEnabled\022\".hbase" +
+      ".pb.IsBalancerEnabledRequest\032#.hbase.pb." +
+      "IsBalancerEnabledResponse\022D\n\tNormalize\022\032" +
+      ".hbase.pb.NormalizeRequest\032\033.hbase.pb.No" +
+      "rmalizeResponse\022e\n\024SetNormalizerRunning\022" +
+      "%.hbase.pb.SetNormalizerRunningRequest\032&" +
+      ".hbase.pb.SetNormalizerRunningResponse\022b" +
+      "\n\023IsNormalizerEnabled\022$.hbase.pb.IsNorma" +
+      "lizerEnabledRequest\032%.hbase.pb.IsNormali",
+      "zerEnabledResponse\022S\n\016RunCatalogScan\022\037.h" +
+      "base.pb.RunCatalogScanRequest\032 .hbase.pb" +
+      ".RunCatalogScanResponse\022e\n\024EnableCatalog" +
+      "Janitor\022%.hbase.pb.EnableCatalogJanitorR" +
+      "equest\032&.hbase.pb.EnableCatalogJanitorRe" +
+      "sponse\022n\n\027IsCatalogJanitorEnabled\022(.hbas" +
+      "e.pb.IsCatalogJanitorEnabledRequest\032).hb" +
+      "ase.pb.IsCatalogJanitorEnabledResponse\022^" +
+      "\n\021ExecMasterService\022#.hbase.pb.Coprocess" +
+      "orServiceRequest\032$.hbase.pb.CoprocessorS",
+      "erviceResponse\022A\n\010Snapshot\022\031.hbase.pb.Sn" +
+      "apshotRequest\032\032.hbase.pb.SnapshotRespons" +
+      "e\022h\n\025GetCompletedSnapshots\022&.hbase.pb.Ge" +
+      "tCompletedSnapshotsRequest\032\'.hbase.pb.Ge" +
+      "tCompletedSnapshotsResponse\022S\n\016DeleteSna" +
+      "pshot\022\037.hbase.pb.DeleteSnapshotRequest\032 " +
+      ".hbase.pb.DeleteSnapshotResponse\022S\n\016IsSn" +
+      "apshotDone\022\037.hbase.pb.IsSnapshotDoneRequ" +
+      "est\032 .hbase.pb.IsSnapshotDoneResponse\022V\n" +
+      "\017RestoreSnapshot\022 .hbase.pb.RestoreSnaps",
+      "hotRequest\032!.hbase.pb.RestoreSnapshotRes" +
+      "ponse\022h\n\025IsRestoreSnapshotDone\022&.hbase.p" +
+      "b.IsRestoreSnapshotDoneRequest\032\'.hbase.p" +
+      "b.IsRestoreSnapshotDoneResponse\022P\n\rExecP" +
+      "rocedure\022\036.hbase.pb.ExecProcedureRequest" +
+      "\032\037.hbase.pb.ExecProcedureResponse\022W\n\024Exe" +
+      "cProcedureWithRet\022\036.hbase.pb.ExecProcedu" +
+      "reRequest\032\037.hbase.pb.ExecProcedureRespon" +
+      "se\022V\n\017IsProcedureDone\022 .hbase.pb.IsProce" +
+      "dureDoneRequest\032!.hbase.pb.IsProcedureDo",
+      "neResponse\022V\n\017ModifyNamespace\022 .hbase.pb" +
+      ".ModifyNamespaceRequest\032!.hbase.pb.Modif" +
+      "yNamespaceResponse\022V\n\017CreateNamespace\022 ." +
+      "hbase.pb.CreateNamespaceRequest\032!.hbase." +
+      "pb.CreateNamespaceResponse\022V\n\017DeleteName" +
+      "space\022 .hbase.pb.DeleteNamespaceRequest\032" +
+      "!.hbase.pb.DeleteNamespaceResponse\022k\n\026Ge" +
+      "tNamespaceDescriptor\022\'.hbase.pb.GetNames" +
+      "paceDescriptorRequest\032(.hbase.pb.GetName" +
+      "spaceDescriptorResponse\022q\n\030ListNamespace",
+      "Descriptors\022).hbase.pb.ListNamespaceDesc" +
+      "riptorsRequest\032*.hbase.pb.ListNamespaceD" +
+      "escriptorsResponse\022\206\001\n\037ListTableDescript" +
+      "orsByNamespace\0220.hbase.pb.ListTableDescr" +
+      "iptorsByNamespaceRequest\0321.hbase.pb.List" +
+      "TableDescriptorsByNamespaceResponse\022t\n\031L" +
+      "istTableNamesByNamespace\022*.hbase.pb.List" +
+      "TableNamesByNamespaceRequest\032+.hbase.pb." +
+      "ListTableNamesByNamespaceResponse\022P\n\rGet" +
+      "TableState\022\036.hbase.pb.GetTableStateReque",
+      "st\032\037.hbase.pb.GetTableStateResponse\022A\n\010S" +
+      "etQuota\022\031.hbase.pb.SetQuotaRequest\032\032.hba" +
+      "se.pb.SetQuotaResponse\022x\n\037getLastMajorCo" +
+      "mpactionTimestamp\022).hbase.pb.MajorCompac" +
+      "tionTimestampRequest\032*.hbase.pb.MajorCom" +
+      "pactionTimestampResponse\022\212\001\n(getLastMajo" +
+      "rCompactionTimestampForRegion\0222.hbase.pb" +
+      ".MajorCompactionTimestampForRegionReques" +
+      "t\032*.hbase.pb.MajorCompactionTimestampRes" +
+      "ponse\022_\n\022getProcedureResult\022#.hbase.pb.G",
+      "etProcedureResultRequest\032$.hbase.pb.GetP" +
+      "rocedureResultResponse\022h\n\027getSecurityCap" +
+      "abilities\022%.hbase.pb.SecurityCapabilitie" +
+      "sRequest\032&.hbase.pb.SecurityCapabilities" +
+      "Response\022S\n\016AbortProcedure\022\037.hbase.pb.Ab" +
+      "ortProcedureRequest\032 .hbase.pb.AbortProc" +
+      "edureResponse\022S\n\016ListProcedures\022\037.hbase." +
+      "pb.ListProceduresRequest\032 .hbase.pb.List" +
+      "ProceduresResponseBB\n*org.apache.hadoop." +
+      "hbase.protobuf.generatedB\014MasterProtosH\001",
+      "\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -62325,7 +62615,7 @@ public final class MasterProtos {
           internal_static_hbase_pb_CreateNamespaceResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_CreateNamespaceResponse_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ProcId", });
           internal_static_hbase_pb_DeleteNamespaceRequest_descriptor =
             getDescriptor().getMessageTypes().get(30);
           internal_static_hbase_pb_DeleteNamespaceRequest_fieldAccessorTable = new
@@ -62337,7 +62627,7 @@ public final class MasterProtos {
           internal_static_hbase_pb_DeleteNamespaceResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_DeleteNamespaceResponse_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ProcId", });
           internal_static_hbase_pb_ModifyNamespaceRequest_descriptor =
             getDescriptor().getMessageTypes().get(32);
           internal_static_hbase_pb_ModifyNamespaceRequest_fieldAccessorTable = new
@@ -62349,7 +62639,7 @@ public final class MasterProtos {
           internal_static_hbase_pb_ModifyNamespaceResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ModifyNamespaceResponse_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ProcId", });
           internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor =
             getDescriptor().getMessageTypes().get(34);
           internal_static_hbase_pb_GetNamespaceDescriptorRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index 4d3a2e1..aa31a5e 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -186,6 +186,7 @@ message CreateNamespaceRequest {
 }
 
 message CreateNamespaceResponse {
+  optional uint64 proc_id = 1;
 }
 
 message DeleteNamespaceRequest {
@@ -195,6 +196,7 @@ message DeleteNamespaceRequest {
 }
 
 message DeleteNamespaceResponse {
+  optional uint64 proc_id = 1;
 }
 
 message ModifyNamespaceRequest {
@@ -204,6 +206,7 @@ message ModifyNamespaceRequest {
 }
 
 message ModifyNamespaceResponse {
+  optional uint64 proc_id = 1;
 }
 
 message GetNamespaceDescriptorRequest {

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
index 365c0b8..26454f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
@@ -43,12 +44,20 @@ public interface Server extends Abortable, Stoppable {
   ZooKeeperWatcher getZooKeeper();
 
   /**
-   * Returns a reference to the servers' cluster connection.
+   * Returns a reference to the servers' connection.
    *
    * Important note: this method returns a reference to Connection which is managed
    * by Server itself, so callers must NOT attempt to close connection obtained.
    */
-  ClusterConnection getConnection();
+  Connection getConnection();
+
+  /**
+   * Returns a reference to the servers' cluster connection. Prefer {@link #getConnection()}.
+   *
+   * Important note: this method returns a reference to Connection which is managed
+   * by Server itself, so callers must NOT attempt to close connection obtained.
+   */
+  ClusterConnection getClusterConnection();
 
   /**
    * Returns instance of {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
new file mode 100644
index 0000000..cb3b684
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
@@ -0,0 +1,131 @@
+/**
+ * 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 java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * View and edit the current cluster schema. Use this API making any modification to
+ * namespaces, tables, etc.
+ *
+ * <h2>Implementation Notes</h2>
+ * Nonces are for when operation is non-idempotent to ensure once-only semantic, even
+ * across process failures.
+ */
+// ClusterSchema is introduced to encapsulate schema modification. Currently the different aspects
+// are spread about the code base. This effort is about cleanup, shutting down access, and
+// coalescing common code. In particular, we'd contain filesystem modification. Other
+// benefits are to make all schema modification work the same way (one way to do an operation only
+// rather than the current approach where how an operation is done varies with context) and to make
+// it so clusterschema modification can stand apart from Master to faciliate standalone
+// testing. It is part of the filesystem refactor project that undoes the dependency on a
+// layout in HDFS that mimics our model of tables have regions have column families have files.
+// With this Interface in place, with all modifications going via this route where no filesystem
+// particulars are exposed, redoing our internals will take less effort.
+//
+// Currently ClusterSchema Interface will include namespace and table manipulation. Ideally a
+// form of this Interface will go all the ways down to the file manipulation level but currently
+// TBD.
+//
+// ClusterSchema is private to the Master; only the Master knows current cluster state and has
+// means of editing/altering it.
+//
+// TODO: Remove Server argument when MasterServices are passed.
+// TODO: We return Future<ProcedureInfo> in the below from most methods. It may change to return
+// a ProcedureFuture subsequently.
+@InterfaceAudience.Private
+public interface ClusterSchema {
+  /**
+   * Timeout for cluster operations in milliseconds.
+   */
+  public static final String HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY =
+      "hbase.master.cluster.schema.operation.timeout";
+  /**
+   * Default operation timeout in milliseconds.
+   */
+  public static final int DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT =
+      5 * 60 * 1000;
+
+  /**
+   * For internals use only. Do not use! Provisionally part of this Interface.
+   * Prefer the high-level APIs available elsewhere in this API.
+   * @return Instance of {@link TableNamespaceManager}
+   */
+  // TODO: Remove from here. Keep internal. This Interface is too high-level to host this accessor.
+  TableNamespaceManager getTableNamespaceManager();
+
+  /**
+   * Create a new Namespace.
+   * @param namespaceDescriptor descriptor for new Namespace
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   *    <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return procedure id
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   *    as well as {@link IOException}
+   */
+  long createNamespace(NamespaceDescriptor namespaceDescriptor, long nonceGroup, long nonce)
+  throws IOException;
+
+  /**
+   * Modify an existing Namespace.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   *    <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return procedure id
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   *    as well as {@link IOException}
+   */
+  long modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce)
+  throws IOException;
+
+  /**
+   * Delete an existing Namespace.
+   * Only empty Namespaces (no tables) can be removed.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   *    <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return procedure id
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   *    as well as {@link IOException}
+   */
+  long deleteNamespace(String name, long nonceGroup, long nonce)
+  throws IOException;
+
+  /**
+   * Get a Namespace
+   * @param name Name of the Namespace
+   * @return Namespace descriptor for <code>name</code>
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   *    as well as {@link IOException}
+   */
+  // No Future here because presumption is that the request will go against cached metadata so
+  // return immediately -- no need of running a Procedure.
+  NamespaceDescriptor getNamespace(String name) throws IOException;
+
+  /**
+   * Get all Namespaces
+   * @return All Namespace descriptors
+   */
+  List<NamespaceDescriptor> getNamespaces() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
new file mode 100644
index 0000000..62892b6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
@@ -0,0 +1,37 @@
+/**
+ * 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.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+public class ClusterSchemaException extends HBaseIOException {
+  public ClusterSchemaException(String message) {
+    super(message);
+  }
+
+  public ClusterSchemaException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ClusterSchemaException(Throwable cause) {
+    super(cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
new file mode 100644
index 0000000..43353ba
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
@@ -0,0 +1,27 @@
+/**
+ * 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.hadoop.hbase.Service;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Mixes in ClusterSchema and Service
+ */
+@InterfaceAudience.Private
+public interface ClusterSchemaService extends ClusterSchema, Service {}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
new file mode 100644
index 0000000..0250f36
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
@@ -0,0 +1,131 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.ServiceNotRunningException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
+import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+
+@InterfaceAudience.Private
+class ClusterSchemaServiceImpl implements ClusterSchemaService {
+  private boolean running = false;
+  private final TableNamespaceManager tableNamespaceManager;
+  private final MasterServices masterServices;
+  private final static List<NamespaceDescriptor> EMPTY_NAMESPACE_LIST =
+    Collections.unmodifiableList(new ArrayList<NamespaceDescriptor>(0));
+
+  ClusterSchemaServiceImpl(final MasterServices masterServices) {
+    this.masterServices = masterServices;
+    this.tableNamespaceManager = new TableNamespaceManager(masterServices);
+  }
+
+  // All below are synchronized so consistent view on whether running or not.
+
+  @Override
+  public synchronized boolean isRunning() {
+    return this.running;
+  }
+
+  private synchronized void checkIsRunning() throws ServiceNotRunningException {
+    if (!isRunning()) throw new ServiceNotRunningException();
+  }
+
+  @Override
+  public synchronized void startAndWait() throws IOException {
+    if (isRunning()) throw new IllegalStateException("Already running; cannot double-start.");
+    // Set to running FIRST because tableNamespaceManager start uses this class to do namespace ops
+    this.running = true;
+    this.tableNamespaceManager.start();
+  }
+
+  @Override
+  public synchronized void stopAndWait() throws IOException {
+    checkIsRunning();
+    // You can't stop tableNamespaceManager.
+    this.running = false;
+  }
+
+  @Override
+  public TableNamespaceManager getTableNamespaceManager() {
+    return this.tableNamespaceManager;
+  }
+
+  private long submitProcedure(final Procedure<?> procedure, long nonceGroup,
+      long nonce)
+  throws ServiceNotRunningException {
+    checkIsRunning();
+    ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
+    return pe.submitProcedure(procedure, nonceGroup, nonce);
+  }
+
+  @Override
+  public long createNamespace(NamespaceDescriptor namespaceDescriptor,
+      long nonceGroup, long nonce)
+  throws IOException {
+    return submitProcedure(new CreateNamespaceProcedure(
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+        nonceGroup, nonce);
+  }
+
+  @Override
+  public long modifyNamespace(NamespaceDescriptor namespaceDescriptor,
+      long nonceGroup, long nonce)
+  throws IOException {
+    return submitProcedure(new ModifyNamespaceProcedure(
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+        nonceGroup, nonce);
+  }
+
+  @Override
+  public long deleteNamespace(String name, long nonceGroup, long nonce)
+  throws IOException {
+    return submitProcedure(new DeleteNamespaceProcedure(
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
+        nonceGroup, nonce);
+  }
+
+  @Override
+  public NamespaceDescriptor getNamespace(String name) throws IOException {
+    NamespaceDescriptor nsd = getTableNamespaceManager().get(name);
+    if (nsd == null) throw new NamespaceNotFoundException(name);
+    return nsd;
+  }
+
+  @Override
+  public List<NamespaceDescriptor> getNamespaces() throws IOException {
+    checkIsRunning();
+    Set<NamespaceDescriptor> set = getTableNamespaceManager().list();
+    if (set == null || set.isEmpty()) return EMPTY_NAMESPACE_LIST;
+    List<NamespaceDescriptor> list = new ArrayList<NamespaceDescriptor>(set.size());
+    list.addAll(set);
+    return Collections.unmodifiableList(list);
+  }
+}
\ No newline at end of file


[10/50] [abbrv] hbase git commit: HBASE-15060 Cull TestHFileV2 and HFileWriterFactory

Posted by jm...@apache.org.
HBASE-15060 Cull TestHFileV2 and HFileWriterFactory


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

Branch: refs/heads/trunk
Commit: 92abf8ac5743849d4c32d6abf8fd33f161bab249
Parents: c1b6d47
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Tue Dec 29 17:41:04 2015 -0800
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Wed Dec 30 13:21:54 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/io/hfile/HFile.java |  10 +-
 .../hbase/io/hfile/HFileWriterFactory.java      |  40 ---
 .../hbase/io/hfile/RandomKeyValueUtil.java      | 104 ++++++
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |   6 +-
 .../hbase/io/hfile/TestHFileBlockIndex.java     |  10 +-
 .../TestHFileInlineToRootChunkConversion.java   |   2 +-
 .../hbase/io/hfile/TestHFileWriterV2.java       | 342 -------------------
 .../hbase/io/hfile/TestHFileWriterV3.java       |   9 +-
 .../hfile/TestLazyDataBlockDecompression.java   |   6 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |   4 +-
 .../hfile/TestSeekBeforeWithInlineBlocks.java   |   6 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |   6 +-
 .../regionserver/TestCompoundBloomFilter.java   |  12 +-
 13 files changed, 137 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 35b232a..669ad92 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -241,7 +241,7 @@ public class HFile {
    * This variety of ways to construct writers is used throughout the code, and
    * we want to be able to swap writer implementations.
    */
-  public static abstract class WriterFactory {
+  public static class WriterFactory {
     protected final Configuration conf;
     protected final CacheConfig cacheConf;
     protected FileSystem fs;
@@ -309,12 +309,8 @@ public class HFile {
           else if (LOG.isDebugEnabled()) LOG.debug("Unable to set drop behind on " + path);
         }
       }
-      return createWriter(fs, path, ostream,
-                   comparator, fileContext);
+      return new HFileWriterImpl(conf, cacheConf, path, ostream, comparator, fileContext);
     }
-
-    protected abstract Writer createWriter(FileSystem fs, Path path, FSDataOutputStream ostream,
-        CellComparator comparator, HFileContext fileContext) throws IOException;
   }
 
   /** The configuration key for HFile version to use for new files */
@@ -351,7 +347,7 @@ public class HFile {
         " hfiles only (but it can read v2 files without having to update hfile.format.version " +
         "in hbase-site.xml)");
     case 3:
-      return new HFileWriterFactory(conf, cacheConf);
+      return new HFile.WriterFactory(conf, cacheConf);
     default:
       throw new IllegalArgumentException("Cannot create writer for HFile " +
           "format version " + version);

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterFactory.java
deleted file mode 100644
index 92eb760..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitationsME
- * under the License.
- */
-package org.apache.hadoop.hbase.io.hfile;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CellComparator;
-
-public class HFileWriterFactory extends HFile.WriterFactory {
-  HFileWriterFactory(Configuration conf, CacheConfig cacheConf) {
-    super(conf, cacheConf);
-  }
-
-  @Override
-  public HFile.Writer createWriter(FileSystem fs, Path path, FSDataOutputStream ostream,
-      CellComparator comparator, HFileContext context)
-  throws IOException {
-    return new HFileWriterImpl(conf, cacheConf, path, ostream, comparator, context);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java
new file mode 100644
index 0000000..9507345
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomKeyValueUtil.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.hfile;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+import java.util.Random;
+
+/**
+ * These helper methods generate random byte[]'s data for KeyValues
+ */
+public class RandomKeyValueUtil {
+  public static final String COLUMN_FAMILY_NAME = "_-myColumnFamily-_";
+  private static final int MIN_ROW_OR_QUALIFIER_LENGTH = 64;
+  private static final int MAX_ROW_OR_QUALIFIER_LENGTH = 128;
+
+  public static final char randomReadableChar(Random rand) {
+    int i = rand.nextInt(26 * 2 + 10 + 1);
+    if (i < 26)
+      return (char) ('A' + i);
+    i -= 26;
+
+    if (i < 26)
+      return (char) ('a' + i);
+    i -= 26;
+
+    if (i < 10)
+      return (char) ('0' + i);
+    i -= 10;
+
+    assert i == 0;
+    return '_';
+  }
+
+  public static KeyValue randomKeyValue(Random rand) {
+    return new KeyValue(randomRowOrQualifier(rand),
+        COLUMN_FAMILY_NAME.getBytes(), randomRowOrQualifier(rand),
+        randomValue(rand));
+  }
+
+  public static byte[] randomRowOrQualifier(Random rand) {
+    StringBuilder field = new StringBuilder();
+    int fieldLen = MIN_ROW_OR_QUALIFIER_LENGTH
+        + rand.nextInt(MAX_ROW_OR_QUALIFIER_LENGTH
+        - MIN_ROW_OR_QUALIFIER_LENGTH + 1);
+    for (int i = 0; i < fieldLen; ++i)
+      field.append(randomReadableChar(rand));
+    return field.toString().getBytes();
+  }
+
+  public static byte[] randomValue(Random rand) {
+    StringBuilder v = new StringBuilder();
+    for (int j = 0; j < 1 + rand.nextInt(2000); ++j) {
+      v.append((char) (32 + rand.nextInt(95)));
+    }
+
+    byte[] valueBytes = v.toString().getBytes();
+    return valueBytes;
+  }
+
+  /**
+   * Generates a random key that is guaranteed to increase as the given index i
+   * increases. The result consists of a prefix, which is a deterministic
+   * increasing function of i, and a random suffix.
+   *
+   * @param rand
+   *          random number generator to use
+   * @param i
+   * @return
+   */
+  public static byte[] randomOrderedKey(Random rand, int i) {
+    StringBuilder k = new StringBuilder();
+
+    // The fixed-length lexicographically increasing part of the key.
+    for (int bitIndex = 31; bitIndex >= 0; --bitIndex) {
+      if ((i & (1 << bitIndex)) == 0)
+        k.append("a");
+      else
+        k.append("b");
+    }
+
+    // A random-length random suffix of the key.
+    for (int j = 0; j < rand.nextInt(50); ++j)
+      k.append(randomReadableChar(rand));
+
+    byte[] keyBytes = k.toString().getBytes();
+    return keyBytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index 2c957ef..ccf59a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -380,9 +380,9 @@ public class TestCacheOnWrite {
         .withBloomType(BLOOM_TYPE).withMaxKeyCount(NUM_KV).build();
     byte[] cf = Bytes.toBytes("fam");
     for (int i = 0; i < NUM_KV; ++i) {
-      byte[] row = TestHFileWriterV2.randomOrderedKey(rand, i);
-      byte[] qualifier = TestHFileWriterV2.randomRowOrQualifier(rand);
-      byte[] value = TestHFileWriterV2.randomValue(rand);
+      byte[] row = RandomKeyValueUtil.randomOrderedKey(rand, i);
+      byte[] qualifier = RandomKeyValueUtil.randomRowOrQualifier(rand);
+      byte[] value = RandomKeyValueUtil.randomValue(rand);
       KeyValue kv;
       if(useTags) {
         Tag t = new Tag((byte) 1, "visibility");

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index 8a843a3..687d3cd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -280,7 +280,7 @@ public class TestHFileBlockIndex {
       byte[] qualifier = Bytes.toBytes("q");
       for (int j = 0; j < 16; ++j) {
         byte[] k =
-            new KeyValue(TestHFileWriterV2.randomOrderedKey(rand, i * 16 + j), family, qualifier,
+            new KeyValue(RandomKeyValueUtil.randomOrderedKey(rand, i * 16 + j), family, qualifier,
                 EnvironmentEdgeManager.currentTime(), KeyValue.Type.Put).getKey();
         keys.add(k);
         if (j == 8) {
@@ -348,7 +348,7 @@ public class TestHFileBlockIndex {
     int secondaryIndexEntries[] = new int[numTotalKeys];
 
     for (int i = 0; i < numTotalKeys; ++i) {
-      byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i * 2);
+      byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i * 2);
       KeyValue cell = new KeyValue(k, Bytes.toBytes("f"), Bytes.toBytes("q"), 
           Bytes.toBytes("val"));
       //KeyValue cell = new KeyValue.KeyOnlyKeyValue(k, 0, k.length);
@@ -473,7 +473,7 @@ public class TestHFileBlockIndex {
       c.writeRoot(dos);
       assertEquals(c.getRootSize(), dos.size());
 
-      byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i);
+      byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i);
       numSubEntries += rand.nextInt(5) + 1;
       keys.add(k);
       c.add(k, getDummyFileOffset(i), getDummyOnDiskSize(i), numSubEntries);
@@ -556,12 +556,12 @@ public class TestHFileBlockIndex {
         byte[] family = Bytes.toBytes("f");
         byte[] qualifier = Bytes.toBytes("q");
         for (int i = 0; i < NUM_KV; ++i) {
-          byte[] row = TestHFileWriterV2.randomOrderedKey(rand, i);
+          byte[] row = RandomKeyValueUtil.randomOrderedKey(rand, i);
 
           // Key will be interpreted by KeyValue.KEY_COMPARATOR
           KeyValue kv =
               new KeyValue(row, family, qualifier, EnvironmentEdgeManager.currentTime(),
-                  TestHFileWriterV2.randomValue(rand));
+                  RandomKeyValueUtil.randomValue(rand));
           byte[] k = kv.getKey();
           writer.append(kv);
           keys[i] = k;

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
index ab811f1..af4f2b8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java
@@ -55,7 +55,7 @@ public class TestHFileInlineToRootChunkConversion {
     CacheConfig cacheConf = new CacheConfig(conf);
     conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize); 
     HFileContext context = new HFileContextBuilder().withBlockSize(16).build();
-    HFile.Writer hfw = new HFileWriterFactory(conf, cacheConf)
+    HFile.Writer hfw = new HFile.WriterFactory(conf, cacheConf)
             .withFileContext(context)
             .withPath(fs, hfPath).create();
     List<byte[]> keys = new ArrayList<byte[]>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
deleted file mode 100644
index 975073a..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
+++ /dev/null
@@ -1,342 +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.io.hfile;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
-import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
-import org.apache.hadoop.hbase.nio.ByteBuff;
-import org.apache.hadoop.hbase.testclassification.IOTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Testing writing a version 2 {@link HFile}. This is a low-level test written
- * during the development of {@link HFileWriterImpl}.
- */
-@Category({IOTests.class, SmallTests.class})
-public class TestHFileWriterV2 {
-
-  private static final Log LOG = LogFactory.getLog(TestHFileWriterV2.class);
-
-  private static final HBaseTestingUtility TEST_UTIL =
-      new HBaseTestingUtility();
-
-  private Configuration conf;
-  private FileSystem fs;
-
-  @Before
-  public void setUp() throws IOException {
-    conf = TEST_UTIL.getConfiguration();
-    fs = FileSystem.get(conf);
-  }
-
-  @Test
-  public void testHFileFormatV2() throws IOException {
-    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), "testHFileFormatV2");
-    final Compression.Algorithm compressAlgo = Compression.Algorithm.GZ;
-    final int entryCount = 10000;
-    writeDataAndReadFromHFile(hfilePath, compressAlgo, entryCount, false);
-  }
-
-  @Test
-  public void testMidKeyInHFile() throws IOException{
-    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
-    "testMidKeyInHFile");
-    Compression.Algorithm compressAlgo = Compression.Algorithm.NONE;
-    int entryCount = 50000;
-    writeDataAndReadFromHFile(hfilePath, compressAlgo, entryCount, true);
-  }
-
-  private void writeDataAndReadFromHFile(Path hfilePath,
-      Algorithm compressAlgo, int entryCount, boolean findMidKey) throws IOException {
-
-    HFileContext context = new HFileContextBuilder()
-                           .withBlockSize(4096)
-                           .withCompression(compressAlgo)
-                           .build();
-    HFile.Writer writer = new HFileWriterFactory(conf, new CacheConfig(conf))
-            .withPath(fs, hfilePath)
-            .withFileContext(context)
-            .create();
-
-    Random rand = new Random(9713312); // Just a fixed seed.
-    List<KeyValue> keyValues = new ArrayList<KeyValue>(entryCount);
-
-    for (int i = 0; i < entryCount; ++i) {
-      byte[] keyBytes = randomOrderedKey(rand, i);
-
-      // A random-length random value.
-      byte[] valueBytes = randomValue(rand);
-      KeyValue keyValue = new KeyValue(keyBytes, null, null, valueBytes);
-      writer.append(keyValue);
-      keyValues.add(keyValue);
-    }
-
-    // Add in an arbitrary order. They will be sorted lexicographically by
-    // the key.
-    writer.appendMetaBlock("CAPITAL_OF_USA", new Text("Washington, D.C."));
-    writer.appendMetaBlock("CAPITAL_OF_RUSSIA", new Text("Moscow"));
-    writer.appendMetaBlock("CAPITAL_OF_FRANCE", new Text("Paris"));
-
-    writer.close();
-
-
-    FSDataInputStream fsdis = fs.open(hfilePath);
-
-    // A "manual" version of a new-format HFile reader. This unit test was
-    // written before the V2 reader was fully implemented.
-
-    long fileSize = fs.getFileStatus(hfilePath).getLen();
-    FixedFileTrailer trailer =
-        FixedFileTrailer.readFromStream(fsdis, fileSize);
-
-    assertEquals(entryCount, trailer.getEntryCount());
-
-    HFileContext meta = new HFileContextBuilder()
-                        .withHBaseCheckSum(true)
-                        .withIncludesMvcc(false)
-                        .withIncludesTags(false)
-                        .withCompression(compressAlgo)
-                        .build();
-
-    HFileBlock.FSReader blockReader = new HFileBlock.FSReaderImpl(fsdis, fileSize, meta);
-    // Comparator class name is stored in the trailer in version 2.
-    CellComparator comparator = trailer.createComparator();
-    HFileBlockIndex.BlockIndexReader dataBlockIndexReader =
-        new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator,
-            trailer.getNumDataIndexLevels());
-    HFileBlockIndex.BlockIndexReader metaBlockIndexReader =
-        new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
-
-    HFileBlock.BlockIterator blockIter = blockReader.blockRange(
-        trailer.getLoadOnOpenDataOffset(),
-        fileSize - trailer.getTrailerSize());
-    // Data index. We also read statistics about the block index written after
-    // the root level.
-    dataBlockIndexReader.readMultiLevelIndexRoot(
-        blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX),
-        trailer.getDataIndexCount());
-
-    if (findMidKey) {
-      Cell midkey = dataBlockIndexReader.midkey();
-      assertNotNull("Midkey should not be null", midkey);
-    }
-
-    // Meta index.
-    metaBlockIndexReader.readRootIndex(
-        blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX)
-          .getByteStream(), trailer.getMetaIndexCount());
-    // File info
-    FileInfo fileInfo = new FileInfo();
-    fileInfo.read(blockIter.nextBlockWithBlockType(BlockType.FILE_INFO).getByteStream());
-    byte [] keyValueFormatVersion = fileInfo.get(HFileWriterImpl.KEY_VALUE_VERSION);
-    boolean includeMemstoreTS = keyValueFormatVersion != null &&
-        Bytes.toInt(keyValueFormatVersion) > 0;
-
-    // Counters for the number of key/value pairs and the number of blocks
-    int entriesRead = 0;
-    int blocksRead = 0;
-    long memstoreTS = 0;
-
-    // Scan blocks the way the reader would scan them
-    fsdis.seek(0);
-    long curBlockPos = 0;
-    while (curBlockPos <= trailer.getLastDataBlockOffset()) {
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false);
-      assertEquals(BlockType.DATA, block.getBlockType());
-      if (meta.isCompressedOrEncrypted()) {
-        assertFalse(block.isUnpacked());
-        block = block.unpack(meta, blockReader);
-      }
-      ByteBuff buf = block.getBufferWithoutHeader();
-      while (buf.hasRemaining()) {
-        int keyLen = buf.getInt();
-        int valueLen = buf.getInt();
-
-        byte[] key = new byte[keyLen];
-        buf.get(key);
-
-        byte[] value = new byte[valueLen];
-        buf.get(value);
-
-        if (includeMemstoreTS) {
-          ByteArrayInputStream byte_input = new ByteArrayInputStream(buf.array(),
-                               buf.arrayOffset() + buf.position(), buf.remaining());
-          DataInputStream data_input = new DataInputStream(byte_input);
-
-          memstoreTS = WritableUtils.readVLong(data_input);
-          buf.position(buf.position() + WritableUtils.getVIntSize(memstoreTS));
-        }
-
-        // A brute-force check to see that all keys and values are correct.
-        KeyValue kv = keyValues.get(entriesRead);
-        assertTrue(Bytes.compareTo(key, kv.getKey()) == 0);
-        assertTrue(Bytes.compareTo(value, 0, value.length, kv.getValueArray(), kv.getValueOffset(),
-          kv.getValueLength()) == 0);
-
-        ++entriesRead;
-      }
-      ++blocksRead;
-      curBlockPos += block.getOnDiskSizeWithHeader();
-    }
-    LOG.info("Finished reading: entries=" + entriesRead + ", blocksRead="
-        + blocksRead);
-    assertEquals(entryCount, entriesRead);
-
-    // Meta blocks. We can scan until the load-on-open data offset (which is
-    // the root block index offset in version 2) because we are not testing
-    // intermediate-level index blocks here.
-
-    int metaCounter = 0;
-    while (fsdis.getPos() < trailer.getLoadOnOpenDataOffset()) {
-      LOG.info("Current offset: " + fsdis.getPos() + ", scanning until " +
-          trailer.getLoadOnOpenDataOffset());
-      HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false)
-        .unpack(meta, blockReader);
-      assertEquals(BlockType.META, block.getBlockType());
-      Text t = new Text();
-      ByteBuff buf = block.getBufferWithoutHeader();
-      if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) {
-        throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName());
-      }
-      Text expectedText =
-          (metaCounter == 0 ? new Text("Paris") : metaCounter == 1 ? new Text(
-              "Moscow") : new Text("Washington, D.C."));
-      assertEquals(expectedText, t);
-      LOG.info("Read meta block data: " + t);
-      ++metaCounter;
-      curBlockPos += block.getOnDiskSizeWithHeader();
-    }
-
-    fsdis.close();
-  }
-
-
-  // Static stuff used by various HFile v2 unit tests
-
-  public static final String COLUMN_FAMILY_NAME = "_-myColumnFamily-_";
-  private static final int MIN_ROW_OR_QUALIFIER_LENGTH = 64;
-  private static final int MAX_ROW_OR_QUALIFIER_LENGTH = 128;
-
-  /**
-   * Generates a random key that is guaranteed to increase as the given index i
-   * increases. The result consists of a prefix, which is a deterministic
-   * increasing function of i, and a random suffix.
-   *
-   * @param rand
-   *          random number generator to use
-   * @param i
-   * @return
-   */
-  public static byte[] randomOrderedKey(Random rand, int i) {
-    StringBuilder k = new StringBuilder();
-
-    // The fixed-length lexicographically increasing part of the key.
-    for (int bitIndex = 31; bitIndex >= 0; --bitIndex) {
-      if ((i & (1 << bitIndex)) == 0)
-        k.append("a");
-      else
-        k.append("b");
-    }
-
-    // A random-length random suffix of the key.
-    for (int j = 0; j < rand.nextInt(50); ++j)
-      k.append(randomReadableChar(rand));
-
-    byte[] keyBytes = k.toString().getBytes();
-    return keyBytes;
-  }
-
-  public static byte[] randomValue(Random rand) {
-    StringBuilder v = new StringBuilder();
-    for (int j = 0; j < 1 + rand.nextInt(2000); ++j) {
-      v.append((char) (32 + rand.nextInt(95)));
-    }
-
-    byte[] valueBytes = v.toString().getBytes();
-    return valueBytes;
-  }
-
-  public static final char randomReadableChar(Random rand) {
-    int i = rand.nextInt(26 * 2 + 10 + 1);
-    if (i < 26)
-      return (char) ('A' + i);
-    i -= 26;
-
-    if (i < 26)
-      return (char) ('a' + i);
-    i -= 26;
-
-    if (i < 10)
-      return (char) ('0' + i);
-    i -= 10;
-
-    assert i == 0;
-    return '_';
-  }
-
-  public static byte[] randomRowOrQualifier(Random rand) {
-    StringBuilder field = new StringBuilder();
-    int fieldLen = MIN_ROW_OR_QUALIFIER_LENGTH
-        + rand.nextInt(MAX_ROW_OR_QUALIFIER_LENGTH
-            - MIN_ROW_OR_QUALIFIER_LENGTH + 1);
-    for (int i = 0; i < fieldLen; ++i)
-      field.append(randomReadableChar(rand));
-    return field.toString().getBytes();
-  }
-
-  public static KeyValue randomKeyValue(Random rand) {
-    return new KeyValue(randomRowOrQualifier(rand),
-        COLUMN_FAMILY_NAME.getBytes(), randomRowOrQualifier(rand),
-        randomValue(rand));
-  }
-
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index 979c9f6..15aa912 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -120,7 +120,7 @@ public class TestHFileWriterV3 {
                            .withBlockSize(4096)
                            .withIncludesTags(useTags)
                            .withCompression(compressAlgo).build();
-    HFile.Writer writer = new HFileWriterFactory(conf, new CacheConfig(conf))
+    HFile.Writer writer = new HFile.WriterFactory(conf, new CacheConfig(conf))
             .withPath(fs, hfilePath)
             .withFileContext(context)
             .withComparator(CellComparator.COMPARATOR)
@@ -130,10 +130,10 @@ public class TestHFileWriterV3 {
     List<KeyValue> keyValues = new ArrayList<KeyValue>(entryCount);
 
     for (int i = 0; i < entryCount; ++i) {
-      byte[] keyBytes = TestHFileWriterV2.randomOrderedKey(rand, i);
+      byte[] keyBytes = RandomKeyValueUtil.randomOrderedKey(rand, i);
 
       // A random-length random value.
-      byte[] valueBytes = TestHFileWriterV2.randomValue(rand);
+      byte[] valueBytes = RandomKeyValueUtil.randomValue(rand);
       KeyValue keyValue = null;
       if (useTags) {
         ArrayList<Tag> tags = new ArrayList<Tag>();
@@ -176,7 +176,7 @@ public class TestHFileWriterV3 {
                         .withHBaseCheckSum(true).build();
     HFileBlock.FSReader blockReader =
         new HFileBlock.FSReaderImpl(fsdis, fileSize, meta);
- // Comparator class name is stored in the trailer in version 2.
+    // Comparator class name is stored in the trailer in version 3.
     CellComparator comparator = trailer.createComparator();
     HFileBlockIndex.BlockIndexReader dataBlockIndexReader =
         new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator,
@@ -297,6 +297,5 @@ public class TestHFileWriterV3 {
 
     fsdis.close();
   }
-
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
index 0067417..5f73500 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
@@ -89,7 +89,7 @@ public class TestLazyDataBlockDecompression {
    */
   private static void writeHFile(Configuration conf, CacheConfig cc, FileSystem fs, Path path,
       HFileContext cxt, int entryCount) throws IOException {
-    HFile.Writer writer = new HFileWriterFactory(conf, cc)
+    HFile.Writer writer = new HFile.WriterFactory(conf, cc)
         .withPath(fs, path)
         .withFileContext(cxt)
         .create();
@@ -100,8 +100,8 @@ public class TestLazyDataBlockDecompression {
     final byte[] qualifier = Bytes.toBytes("q");
 
     for (int i = 0; i < entryCount; i++) {
-      byte[] keyBytes = TestHFileWriterV2.randomOrderedKey(rand, i);
-      byte[] valueBytes = TestHFileWriterV2.randomValue(rand);
+      byte[] keyBytes = RandomKeyValueUtil.randomOrderedKey(rand, i);
+      byte[] valueBytes = RandomKeyValueUtil.randomValue(rand);
       // make a real keyvalue so that hfile tool can examine it
       writer.append(new KeyValue(keyBytes, family, qualifier, valueBytes));
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
index 053a27e..69a77bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
@@ -111,8 +111,8 @@ public class TestPrefetch {
 
     final int rowLen = 32;
     for (int i = 0; i < NUM_KV; ++i) {
-      byte[] k = TestHFileWriterV2.randomOrderedKey(RNG, i);
-      byte[] v = TestHFileWriterV2.randomValue(RNG);
+      byte[] k = RandomKeyValueUtil.randomOrderedKey(RNG, i);
+      byte[] v = RandomKeyValueUtil.randomValue(RNG);
       int cfLen = RNG.nextInt(k.length - rowLen + 1);
       KeyValue kv = new KeyValue(
           k, 0, rowLen,

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
index c9af3d3..0dbb4e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
@@ -123,9 +123,9 @@ public class TestSeekBeforeWithInlineBlocks {
               .build();
             
             for (int i = 0; i < NUM_KV; i++) {
-              byte[] row = TestHFileWriterV2.randomOrderedKey(RAND, i);
-              byte[] qual = TestHFileWriterV2.randomRowOrQualifier(RAND);
-              byte[] value = TestHFileWriterV2.randomValue(RAND);
+              byte[] row = RandomKeyValueUtil.randomOrderedKey(RAND, i);
+              byte[] qual = RandomKeyValueUtil.randomRowOrQualifier(RAND);
+              byte[] value = RandomKeyValueUtil.randomValue(RAND);
               KeyValue kv = new KeyValue(row, FAM, qual, value);
   
               storeFileWriter.append(kv);

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index da3c17d..0916fe6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.TableName;
@@ -49,7 +50,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileBlock;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -276,8 +276,8 @@ public class TestCacheOnWriteInSchema {
   private void writeStoreFile(StoreFile.Writer writer) throws IOException {
     final int rowLen = 32;
     for (int i = 0; i < NUM_KV; ++i) {
-      byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i);
-      byte[] v = TestHFileWriterV2.randomValue(rand);
+      byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i);
+      byte[] v = RandomKeyValueUtil.randomValue(rand);
       int cfLen = rand.nextInt(k.length - rowLen + 1);
       KeyValue kv = new KeyValue(
           k, 0, rowLen,

http://git-wip-us.apache.org/repos/asf/hbase/blob/92abf8ac/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
index 8300210..44e249d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.client.Scan;
@@ -53,7 +54,6 @@ import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
@@ -144,7 +144,7 @@ public class TestCompoundBloomFilter {
   private List<KeyValue> createSortedKeyValues(Random rand, int n) {
     List<KeyValue> kvList = new ArrayList<KeyValue>(n);
     for (int i = 0; i < n; ++i)
-      kvList.add(TestHFileWriterV2.randomKeyValue(rand));
+      kvList.add(RandomKeyValueUtil.randomKeyValue(rand));
     Collections.sort(kvList, CellComparator.COMPARATOR);
     return kvList;
   }
@@ -231,7 +231,7 @@ public class TestCompoundBloomFilter {
         Random rand = new Random(EVALUATION_SEED);
         int nTrials = NUM_KV[t] * 10;
         for (int i = 0; i < nTrials; ++i) {
-          byte[] query = TestHFileWriterV2.randomRowOrQualifier(rand);
+          byte[] query = RandomKeyValueUtil.randomRowOrQualifier(rand);
           if (isInBloom(scanner, query, bt, rand)) {
             numFalsePos += 1;
           }
@@ -280,16 +280,16 @@ public class TestCompoundBloomFilter {
 
   private boolean isInBloom(StoreFileScanner scanner, byte[] row, BloomType bt,
       Random rand) {
-    return isInBloom(scanner, row, TestHFileWriterV2.randomRowOrQualifier(rand));
+    return isInBloom(scanner, row, RandomKeyValueUtil.randomRowOrQualifier(rand));
   }
 
   private boolean isInBloom(StoreFileScanner scanner, byte[] row,
       byte[] qualifier) {
     Scan scan = new Scan(row, row);
-    scan.addColumn(Bytes.toBytes(TestHFileWriterV2.COLUMN_FAMILY_NAME), qualifier);
+    scan.addColumn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME), qualifier);
     Store store = mock(Store.class);
     HColumnDescriptor hcd = mock(HColumnDescriptor.class);
-    when(hcd.getName()).thenReturn(Bytes.toBytes(TestHFileWriterV2.COLUMN_FAMILY_NAME));
+    when(hcd.getName()).thenReturn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME));
     when(store.getFamily()).thenReturn(hcd);
     return scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
   }


[34/50] [abbrv] hbase git commit: HBASE-15076 Add getScanner(Scan scan, List additionalScanners) API into Region interface (Anoop Sam John)

Posted by jm...@apache.org.
HBASE-15076 Add getScanner(Scan scan, List<KeyValueScanner> additionalScanners) API into Region interface (Anoop Sam John)


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

Branch: refs/heads/trunk
Commit: 5bde960b9525f97d26f8917041d550eeb0e2b781
Parents: 3d36779
Author: stack <st...@apache.org>
Authored: Thu Jan 7 13:45:01 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Jan 7 13:45:01 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 26 ++++++++++++--------
 .../hadoop/hbase/regionserver/Region.java       | 15 +++++++++++
 2 files changed, 31 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5bde960b/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 e553fcc..38404c7 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
@@ -1819,27 +1819,27 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
      * We are trying to remove / relax the region read lock for compaction.
      * Let's see what are the potential race conditions among the operations (user scan,
      * region split, region close and region bulk load).
-     * 
+     *
      *  user scan ---> region read lock
      *  region split --> region close first --> region write lock
      *  region close --> region write lock
      *  region bulk load --> region write lock
-     *  
+     *
      * read lock is compatible with read lock. ---> no problem with user scan/read
      * region bulk load does not cause problem for compaction (no consistency problem, store lock
      *  will help the store file accounting).
      * They can run almost concurrently at the region level.
-     * 
+     *
      * The only remaining race condition is between the region close and compaction.
      * So we will evaluate, below, how region close intervenes with compaction if compaction does
      * not acquire region read lock.
-     * 
+     *
      * Here are the steps for compaction:
      * 1. obtain list of StoreFile's
      * 2. create StoreFileScanner's based on list from #1
      * 3. perform compaction and save resulting files under tmp dir
      * 4. swap in compacted files
-     * 
+     *
      * #1 is guarded by store lock. This patch does not change this --> no worse or better
      * For #2, we obtain smallest read point (for region) across all the Scanners (for both default
      * compactor and stripe compactor).
@@ -1851,7 +1851,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
      * This will not conflict with compaction.
      * For #3, it can be performed in parallel to other operations.
      * For #4 bulk load and compaction don't conflict with each other on the region level
-     *   (for multi-family atomicy). 
+     *   (for multi-family atomicy).
      * Region close and compaction are guarded pretty well by the 'writestate'.
      * In HRegion#doClose(), we have :
      * synchronized (writestate) {
@@ -2575,6 +2575,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    return getScanner(scan, true);
   }
 
+  @Override
+  public RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners)
+      throws IOException {
+    return getScanner(scan, additionalScanners, true);
+  }
+
   public RegionScanner getScanner(Scan scan, boolean copyCellsFromSharedMem) throws IOException {
     RegionScanner scanner = getScanner(scan, null, copyCellsFromSharedMem);
     return scanner;
@@ -7073,7 +7079,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
     if (cell.getTagsLength() <= 0) return tags;
-    List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags; 
+    List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags;
     Iterator<Tag> i = CellUtil.tagsIterator(cell);
     while (i.hasNext()) newTags.add(i.next());
     return newTags;
@@ -7349,7 +7355,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   // They are subtley different in quiet a few ways. This came out only
   // after study. I am not sure that many of the differences are intentional.
-  // TODO: St.Ack 20150907 
+  // TODO: St.Ack 20150907
 
   @Override
   public Result increment(Increment mutation, long nonceGroup, long nonce)
@@ -7363,7 +7369,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     boolean writeToWAL = durability != Durability.SKIP_WAL;
     WALEdit walEdits = null;
     List<Cell> allKVs = new ArrayList<Cell>(mutation.size());
-    
+
     Map<Store, List<Cell>> tempMemstore = new HashMap<Store, List<Cell>>();
     long size = 0;
     long txid = 0;
@@ -8166,7 +8172,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     WALKey key = new HLogKey(getRegionInfo().getEncodedNameAsBytes(),
       getRegionInfo().getTable(), WALKey.NO_SEQUENCE_ID, 0, null,
       HConstants.NO_NONCE, HConstants.NO_NONCE, getMVCC());
-    
+
     // Call append but with an empty WALEdit.  The returned sequence id will not be associated
     // with any edit and we can be sure it went in after all outstanding appends.
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5bde960b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 6d87057..5da8bcb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -392,6 +392,21 @@ public interface Region extends ConfigurationObserver {
    */
   RegionScanner getScanner(Scan scan) throws IOException;
 
+  /**
+   * Return an iterator that scans over the HRegion, returning the indicated columns and rows
+   * specified by the {@link Scan}. The scanner will also include the additional scanners passed
+   * along with the scanners for the specified Scan instance. Should be careful with the usage to
+   * pass additional scanners only within this Region
+   * <p>
+   * This Iterator must be closed by the caller.
+   *
+   * @param scan configured {@link Scan}
+   * @param additionalScanners Any additional scanners to be used
+   * @return RegionScanner
+   * @throws IOException read exceptions
+   */
+  RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException;
+
   /** The comparator to be used with the region */
   CellComparator getCellCompartor();
 


[26/50] [abbrv] hbase git commit: HBASE-14221 Reduce the number of time row comparison is done in a Scan (Ram)

Posted by jm...@apache.org.
HBASE-14221 Reduce the number of time row comparison is done in a Scan
(Ram)


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

Branch: refs/heads/trunk
Commit: 073e00c0053cb351dbba265112a6e756c932f875
Parents: 7cd09bf
Author: ramkrishna <ra...@gmail.com>
Authored: Wed Jan 6 10:38:27 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Wed Jan 6 10:38:27 2016 +0530

----------------------------------------------------------------------
 .../hbase/regionserver/ScanQueryMatcher.java    | 36 ++++++++++++--------
 .../hadoop/hbase/regionserver/StoreScanner.java | 11 +++---
 2 files changed, 27 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/073e00c0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index 47d8c8f..c220b5c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -283,27 +283,33 @@ public class ScanQueryMatcher {
       if (filter != null && filter.filterAllRemaining()) {
       return MatchCode.DONE_SCAN;
     }
-    int ret = this.rowComparator.compareRows(curCell, cell);
-    if (!this.isReversed) {
-      if (ret <= -1) {
-        return MatchCode.DONE;
-      } else if (ret >= 1) {
-        // could optimize this, if necessary?
-        // Could also be called SEEK_TO_CURRENT_ROW, but this
-        // should be rare/never happens.
-        return MatchCode.SEEK_NEXT_ROW;
+    if (curCell != null) {
+      int ret = this.rowComparator.compareRows(curCell, cell);
+      if (!this.isReversed) {
+        if (ret <= -1) {
+          return MatchCode.DONE;
+        } else if (ret >= 1) {
+          // could optimize this, if necessary?
+          // Could also be called SEEK_TO_CURRENT_ROW, but this
+          // should be rare/never happens.
+          return MatchCode.SEEK_NEXT_ROW;
+        }
+      } else {
+        if (ret <= -1) {
+          return MatchCode.SEEK_NEXT_ROW;
+        } else if (ret >= 1) {
+          return MatchCode.DONE;
+        }
       }
     } else {
-      if (ret <= -1) {
-        return MatchCode.SEEK_NEXT_ROW;
-      } else if (ret >= 1) {
-        return MatchCode.DONE;
-      }
+      // Since the curCell is null it means we are already sure that we have moved over to the next row
+      return MatchCode.DONE;
     }
 
     // optimize case.
-    if (this.stickyNextRow)
+    if (this.stickyNextRow) {
       return MatchCode.SEEK_NEXT_ROW;
+    }
 
     if (this.columns.done()) {
       stickyNextRow = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/073e00c0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 5fdfa79..3049608 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -507,8 +507,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     // If no limits exists in the scope LimitScope.Between_Cells then we are sure we are changing
     // rows. Else it is possible we are still traversing the same row so we must perform the row
     // comparison.
-    if (!scannerContext.hasAnyLimit(LimitScope.BETWEEN_CELLS) || matcher.curCell == null
-        || !CellUtil.matchingRow(cell, matcher.curCell)) {
+    if (!scannerContext.hasAnyLimit(LimitScope.BETWEEN_CELLS) || matcher.curCell == null) {
       this.countPerRow = 0;
       matcher.setToNewRow(cell);
     }
@@ -534,7 +533,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       if (prevCell != cell) ++kvsScanned; // Do object compare - we set prevKV from the same heap.
       checkScanOrder(prevCell, cell, comparator);
       prevCell = cell;
-
       ScanQueryMatcher.MatchCode qcode = matcher.match(cell);
       qcode = optimize(qcode, cell);
       switch (qcode) {
@@ -553,6 +551,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
           if (!matcher.moreRowsMayExistAfter(cell)) {
             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
           }
+          matcher.curCell = null;
           seekToNextRow(cell);
           break LOOP;
         }
@@ -580,6 +579,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
           if (!matcher.moreRowsMayExistAfter(cell)) {
             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
           }
+          matcher.curCell = null;
           seekToNextRow(cell);
         } else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
           seekAsDirection(matcher.getKeyForNextColumn(cell));
@@ -596,6 +596,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         continue;
 
       case DONE:
+        matcher.curCell = null;
         return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
 
       case DONE_SCAN:
@@ -608,7 +609,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         if (!matcher.moreRowsMayExistAfter(cell)) {
           return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
         }
-
+        matcher.curCell = null;
         seekToNextRow(cell);
         break;
 
@@ -751,7 +752,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     }
     if ((matcher.curCell == null) || !CellUtil.matchingRows(cell, matcher.curCell)) {
       this.countPerRow = 0;
-      matcher.reset();
+      // The setToNewRow will call reset internally
       matcher.setToNewRow(cell);
     }
   }


[18/50] [abbrv] hbase git commit: Revert "HBASE-14902 Revert some of the stringency recently introduced by checkstyle tightening"

Posted by jm...@apache.org.
Revert "HBASE-14902 Revert some of the stringency recently introduced by checkstyle tightening"

This reverts commit 998b9371c9d9a6fe4a93e734ba0055c3e20e0b3e.


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

Branch: refs/heads/trunk
Commit: 1c4edd2ab702488e21c4929a998c49a4208633fc
Parents: 9997e4e
Author: stack <st...@apache.org>
Authored: Mon Jan 4 20:03:56 2016 -0800
Committer: stack <st...@apache.org>
Committed: Mon Jan 4 20:03:56 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/1c4edd2a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
index 6670182..e7272c5 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
@@ -41,9 +41,7 @@
     http://checkstyle.sourceforge.net/config_blocks.html -->
     <module name="EmptyBlock"/>
     <module name="LeftCurly"/>
-    <module name="NeedBraces">
-      <property name="allowSingleLineStatement" value="true"/>
-    </module>
+    <module name="NeedBraces"/>
 
     <!-- Class Design Checks
     http://checkstyle.sourceforge.net/config_design.html -->
@@ -79,9 +77,7 @@
 
     <!-- Javadoc Checks
     http://checkstyle.sourceforge.net/config_javadoc.html -->
-    <module name="JavadocTagContinuationIndentation">
-      <property name="offset" value="2"/>
-    </module>
+    <module name="JavadocTagContinuationIndentation"/>
     <module name="NonEmptyAtclauseDescription"/>
 
     <!-- Miscellaneous Checks


[30/50] [abbrv] hbase git commit: HBASE-12593 Tags to work with ByteBuffer.

Posted by jm...@apache.org.
HBASE-12593 Tags to work with ByteBuffer.


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

Branch: refs/heads/trunk
Commit: a9b671b31f07ade8968b42956aa60c722032dcc8
Parents: 893a54c
Author: anoopsjohn <an...@gmail.com>
Authored: Wed Jan 6 21:28:06 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Wed Jan 6 21:28:06 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/Mutation.java    |   9 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  16 +-
 .../org/apache/hadoop/hbase/ArrayBackedTag.java | 143 ++++++++++++
 .../java/org/apache/hadoop/hbase/CellUtil.java  | 120 +++++++++-
 .../java/org/apache/hadoop/hbase/KeyValue.java  |  22 +-
 .../org/apache/hadoop/hbase/OffheapTag.java     |  83 +++++++
 .../main/java/org/apache/hadoop/hbase/Tag.java  | 191 +++-------------
 .../java/org/apache/hadoop/hbase/TagUtil.java   | 219 +++++++++++++++++++
 .../hadoop/hbase/io/util/StreamUtils.java       |  47 +++-
 .../hadoop/hbase/util/ByteBufferUtils.java      |  23 ++
 .../hbase/util/test/RedundantKVGenerator.java   |   7 +-
 .../org/apache/hadoop/hbase/TestKeyValue.java   |  28 ++-
 .../hadoop/hbase/TestOffheapKeyValue.java       |  25 +--
 .../hbase/codec/TestCellCodecWithTags.java      |  32 +--
 .../hbase/codec/TestKeyValueCodecWithTags.java  |  32 +--
 .../hbase/io/TestTagCompressionContext.java     |   3 +-
 .../util/TestByteRangeWithKVSerialization.java  |   3 +-
 .../row/data/TestRowDataTrivialWithTags.java    |   5 +-
 .../hbase/rest/PerformanceEvaluation.java       |   5 +-
 .../hbase/io/hfile/HFilePrettyPrinter.java      |  10 +-
 .../hadoop/hbase/mapreduce/TextSortReducer.java |   3 +-
 .../hbase/mapreduce/TsvImporterMapper.java      |   3 +-
 .../hbase/mob/DefaultMobStoreCompactor.java     |   4 +-
 .../hbase/mob/DefaultMobStoreFlusher.java       |   5 +-
 .../apache/hadoop/hbase/mob/MobConstants.java   |   3 +-
 .../org/apache/hadoop/hbase/mob/MobUtils.java   |  10 +-
 .../compactions/PartitionedMobCompactor.java    |   3 +-
 .../hbase/mob/mapreduce/MemStoreWrapper.java    |   5 +-
 .../hadoop/hbase/regionserver/HMobStore.java    |   6 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  25 ++-
 .../hadoop/hbase/regionserver/HStore.java       |  39 ++--
 .../security/access/AccessControlLists.java     |  11 +-
 .../hbase/security/access/AccessController.java |  71 +++---
 .../DefaultVisibilityLabelServiceImpl.java      |  82 ++++---
 .../visibility/VisibilityController.java        |  66 +++---
 .../VisibilityReplicationEndpoint.java          |   7 +-
 .../security/visibility/VisibilityUtils.java    |  56 ++---
 .../hadoop/hbase/wal/WALPrettyPrinter.java      |   7 +-
 .../hadoop/hbase/PerformanceEvaluation.java     |   4 +-
 .../hbase/client/TestResultSizeEstimation.java  |   5 +-
 .../io/encoding/TestDataBlockEncoders.java      |  25 ++-
 .../hbase/io/encoding/TestEncodedSeekers.java   |   3 +-
 .../io/encoding/TestPrefixTreeEncoding.java     |   5 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |   5 +-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |   3 +-
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |   5 +-
 .../hbase/io/hfile/TestHFileWriterV3.java       |   3 +-
 .../hadoop/hbase/io/hfile/TestReseekTo.java     |   5 +-
 .../hadoop/hbase/io/hfile/TestSeekTo.java       |  11 +-
 .../hbase/regionserver/TestHMobStore.java       |   5 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  10 +-
 .../TestStoreFileScannerWithTagCompression.java |  12 +-
 .../hadoop/hbase/regionserver/TestTags.java     |  22 +-
 .../wal/TestKeyValueCompression.java            |   5 +-
 .../wal/TestWALCellCodecWithCompression.java    |  10 +-
 .../replication/TestReplicationWithTags.java    |   6 +-
 .../security/access/TestAccessController.java   |   3 +-
 .../ExpAsStringVisibilityLabelServiceImpl.java  |  49 +++--
 ...sibilityLabelReplicationWithExpAsString.java |   3 +-
 .../TestVisibilityLabelsReplication.java        |  16 +-
 .../apache/hadoop/hbase/util/HFileTestUtil.java |  17 +-
 .../util/LoadTestDataGeneratorWithTags.java     |   3 +-
 62 files changed, 1083 insertions(+), 581 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index 665c59c..9a550f9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -124,7 +125,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
    * @param qualifier
    * @param ts
    * @param value
-   * @param tags - Specify the Tags as an Array {@link KeyValue.Tag}
+   * @param tags - Specify the Tags as an Array
    * @return a KeyValue with this objects row key and the Put identifier.
    */
   KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts, byte[] value, Tag[] tags) {
@@ -138,7 +139,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
    * @return a KeyValue with this objects row key and the Put identifier.
    */
   KeyValue createPutKeyValue(byte[] family, ByteBuffer qualifier, long ts, ByteBuffer value,
-                             Tag[] tags) {
+      Tag[] tags) {
     return new KeyValue(this.row, 0, this.row == null ? 0 : this.row.length,
         family, 0, family == null ? 0 : family.length,
         qualifier, ts, KeyValue.Type.Put, value, tags != null ? Arrays.asList(tags) : null);
@@ -219,11 +220,11 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
                 c.getQualifierLength()));
     stringMap.put("timestamp", c.getTimestamp());
     stringMap.put("vlen", c.getValueLength());
-    List<Tag> tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    List<Tag> tags = CellUtil.getTags(c);
     if (tags != null) {
       List<String> tagsString = new ArrayList<String>();
       for (Tag t : tags) {
-        tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(t.getValue()));
+        tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(TagUtil.cloneValue(t)));
       }
       stringMap.put("tag", tagsString);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index c02309b..f5e4305 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Consistency;
@@ -580,20 +581,17 @@ public final class ProtobufUtil {
           if (qv.hasTimestamp()) {
             ts = qv.getTimestamp();
           }
-          byte[] tags;
+          byte[] allTagsBytes;
           if (qv.hasTags()) {
-            tags = qv.getTags().toByteArray();
-            Object[] array = Tag.asList(tags, 0, (short)tags.length).toArray();
-            Tag[] tagArray = new Tag[array.length];
-            for(int i = 0; i< array.length; i++) {
-              tagArray[i] = (Tag)array[i];
-            }
+            allTagsBytes = qv.getTags().toByteArray();
             if(qv.hasDeleteType()) {
               byte[] qual = qv.hasQualifier() ? qv.getQualifier().toByteArray() : null;
               put.add(new KeyValue(proto.getRow().toByteArray(), family, qual, ts,
-                  fromDeleteType(qv.getDeleteType()), null, tags));
+                  fromDeleteType(qv.getDeleteType()), null, allTagsBytes));
             } else {
-              put.addImmutable(family, qualifier, ts, value, tagArray);
+              List<Tag> tags = TagUtil.asList(allTagsBytes, 0, (short)allTagsBytes.length);
+              Tag[] tagsArray = new Tag[tags.size()];
+              put.addImmutable(family, qualifier, ts, value, tags.toArray(tagsArray));
             }
           } else {
             if(qv.hasDeleteType()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/ArrayBackedTag.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ArrayBackedTag.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ArrayBackedTag.java
new file mode 100644
index 0000000..2f4bb75
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ArrayBackedTag.java
@@ -0,0 +1,143 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * This is a {@link Tag} implementation in which value is backed by an on heap byte array.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ArrayBackedTag implements Tag {
+  private final byte type;// TODO  extra type state needed?
+  private final byte[] bytes;
+  private int offset = 0;
+  private int length = 0;
+
+  /**
+   * The special tag will write the length of each tag and that will be
+   * followed by the type and then the actual tag.
+   * So every time the length part is parsed we need to add + 1 byte to it to
+   * get the type and then get the actual tag.
+   */
+  public ArrayBackedTag(byte tagType, String tag) {
+    this(tagType, Bytes.toBytes(tag));
+  }
+
+  /**
+   * Format for a tag :
+   * {@code <length of tag - 2 bytes><type code - 1 byte><tag>} tag length is serialized
+   * using 2 bytes only but as this will be unsigned, we can have max tag length of
+   * (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length.
+   */
+  public ArrayBackedTag(byte tagType, byte[] tag) {
+    int tagLength = tag.length + TYPE_LENGTH_SIZE;
+    if (tagLength > MAX_TAG_LENGTH) {
+      throw new IllegalArgumentException(
+          "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH);
+    }
+    length = TAG_LENGTH_SIZE + tagLength;
+    bytes = new byte[length];
+    int pos = Bytes.putAsShort(bytes, 0, tagLength);
+    pos = Bytes.putByte(bytes, pos, tagType);
+    Bytes.putBytes(bytes, pos, tag, 0, tag.length);
+    this.type = tagType;
+  }
+
+  /**
+   * Creates a Tag from the specified byte array and offset. Presumes
+   * <code>bytes</code> content starting at <code>offset</code> is formatted as
+   * a Tag blob.
+   * The bytes to include the tag type, tag length and actual tag bytes.
+   * @param offset offset to start of Tag
+   */
+  public ArrayBackedTag(byte[] bytes, int offset) {
+    this(bytes, offset, getLength(bytes, offset));
+  }
+
+  private static int getLength(byte[] bytes, int offset) {
+    return TAG_LENGTH_SIZE + Bytes.readAsInt(bytes, offset, TAG_LENGTH_SIZE);
+  }
+
+  /**
+   * Creates a Tag from the specified byte array, starting at offset, and for length
+   * <code>length</code>. Presumes <code>bytes</code> content starting at <code>offset</code> is
+   * formatted as a Tag blob.
+   */
+  public ArrayBackedTag(byte[] bytes, int offset, int length) {
+    if (length > MAX_TAG_LENGTH) {
+      throw new IllegalArgumentException(
+          "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH);
+    }
+    this.bytes = bytes;
+    this.offset = offset;
+    this.length = length;
+    this.type = bytes[offset + TAG_LENGTH_SIZE];
+  }
+
+  /**
+   * @return The byte array backing this Tag.
+   */
+  public byte[] getValueArray() {
+    return this.bytes;
+  }
+
+  /**
+   * @return the tag type
+   */
+  public byte getType() {
+    return this.type;
+  }
+
+  /**
+   * @return Length of actual tag bytes within the backed buffer
+   */
+  public int getValueLength() {
+    return this.length - INFRASTRUCTURE_SIZE;
+  }
+
+  /**
+   * @return Offset of actual tag bytes within the backed buffer
+   */
+  public int getValueOffset() {
+    return this.offset + INFRASTRUCTURE_SIZE;
+  }
+
+  @Override
+  public boolean hasArray() {
+    return true;
+  }
+
+  @Override
+  public ByteBuffer getValueByteBuffer() {
+    return ByteBuffer.wrap(bytes);
+  }
+
+  @Override
+  public String toString() {
+    return "[Tag type : " + this.type + ", value : "
+        + Bytes.toStringBinary(bytes, getValueOffset(), getValueLength()) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 0d34137..1ec6afa 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.hbase;
 
 import static org.apache.hadoop.hbase.HConstants.EMPTY_BYTE_ARRAY;
+import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
@@ -108,8 +110,8 @@ public final class CellUtil {
 
   /**
    * Returns tag value in a new byte array. If server-side, use
-   * {@link Tag#getBuffer()} with appropriate {@link Tag#getTagOffset()} and
-   * {@link Tag#getTagLength()} instead to save on allocations.
+   * {@link Tag#getValueArray()} with appropriate {@link Tag#getValueOffset()} and
+   * {@link Tag#getValueLength()} instead to save on allocations.
    * @param cell
    * @return tag value in a new byte array.
    */
@@ -749,7 +751,10 @@ public final class CellUtil {
    * @param offset
    * @param length
    * @return iterator for the tags
+   * @deprecated As of 2.0.0 and will be removed in 3.0.0
+   *             Instead use {@link #tagsIterator(Cell)}
    */
+  @Deprecated
   public static Iterator<Tag> tagsIterator(final byte[] tags, final int offset, final int length) {
     return new Iterator<Tag>() {
       private int pos = offset;
@@ -764,7 +769,7 @@ public final class CellUtil {
       public Tag next() {
         if (hasNext()) {
           int curTagLen = Bytes.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
-          Tag tag = new Tag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
+          Tag tag = new ArrayBackedTag(tags, pos, curTagLen + TAG_LENGTH_SIZE);
           this.pos += Bytes.SIZEOF_SHORT + curTagLen;
           return tag;
         }
@@ -778,6 +783,115 @@ public final class CellUtil {
     };
   }
 
+  private static Iterator<Tag> tagsIterator(final ByteBuffer tags, final int offset,
+      final int length) {
+    return new Iterator<Tag>() {
+      private int pos = offset;
+      private int endOffset = offset + length - 1;
+
+      @Override
+      public boolean hasNext() {
+        return this.pos < endOffset;
+      }
+
+      @Override
+      public Tag next() {
+        if (hasNext()) {
+          int curTagLen = ByteBufferUtils.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
+          Tag tag = new OffheapTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
+          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
+          return tag;
+        }
+        return null;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  private static final Iterator<Tag> EMPTY_TAGS_ITR = new Iterator<Tag>() {
+    @Override
+    public boolean hasNext() {
+      return false;
+    }
+
+    @Override
+    public Tag next() {
+      return null;
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  };
+
+  /**
+   * Util method to iterate through the tags in the given cell.
+   *
+   * @param cell The Cell over which tags iterator is needed.
+   * @return iterator for the tags
+   */
+  public static Iterator<Tag> tagsIterator(final Cell cell) {
+    final int tagsLength = cell.getTagsLength();
+    // Save an object allocation where we can
+    if (tagsLength == 0) {
+      return EMPTY_TAGS_ITR;
+    }
+    if (cell instanceof ByteBufferedCell) {
+      return tagsIterator(((ByteBufferedCell) cell).getTagsByteBuffer(),
+          ((ByteBufferedCell) cell).getTagsPosition(), tagsLength);
+    }
+    return tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
+  }
+
+  /**
+   * @param cell The Cell
+   * @return Tags in the given Cell as a List
+   */
+  public static List<Tag> getTags(Cell cell) {
+    List<Tag> tags = new ArrayList<Tag>();
+    Iterator<Tag> tagsItr = tagsIterator(cell);
+    while (tagsItr.hasNext()) {
+      tags.add(tagsItr.next());
+    }
+    return tags;
+  }
+
+  /**
+   * Retrieve Cell's first tag, matching the passed in type
+   *
+   * @param cell The Cell
+   * @param type Type of the Tag to retrieve
+   * @return null if there is no tag of the passed in tag type
+   */
+  public static Tag getTag(Cell cell, byte type){
+    boolean bufferBacked = cell instanceof ByteBufferedCell;
+    int length = cell.getTagsLength();
+    int offset = bufferBacked? ((ByteBufferedCell)cell).getTagsPosition():cell.getTagsOffset();
+    int pos = offset;
+    while (pos < offset + length) {
+      int tagLen;
+      if (bufferBacked) {
+        ByteBuffer tagsBuffer = ((ByteBufferedCell)cell).getTagsByteBuffer();
+        tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
+        if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
+          return new OffheapTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE);
+        }
+      } else {
+        tagLen = Bytes.readAsInt(cell.getTagsArray(), pos, TAG_LENGTH_SIZE);
+        if (cell.getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
+          return new ArrayBackedTag(cell.getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE);
+        }
+      }
+      pos += TAG_LENGTH_SIZE + tagLen;
+    }
+    return null;
+  }
+
   /**
    * Returns true if the first range start1...end1 overlaps with the second range
    * start2...end2, assuming the byte arrays represent row keys

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/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 933dd1d..a30a24c 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
@@ -894,7 +894,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     int tagsLength = 0;
     if (tags != null && tags.length > 0) {
       for (Tag t: tags) {
-        tagsLength += t.getLength();
+        tagsLength += t.getValueLength() + Tag.INFRASTRUCTURE_SIZE;
       }
     }
     checkForTagsLength(tagsLength);
@@ -928,7 +928,11 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tagsLength > 0) {
       pos = Bytes.putAsShort(buffer, pos, tagsLength);
       for (Tag t : tags) {
-        pos = Bytes.putBytes(buffer, pos, t.getBuffer(), t.getOffset(), t.getLength());
+        int tlen = t.getValueLength();
+        pos = Bytes.putAsShort(buffer, pos, tlen + Tag.TYPE_LENGTH_SIZE);
+        pos = Bytes.putByte(buffer, pos, t.getType());
+        TagUtil.copyValueTo(t, buffer, pos);
+        pos += tlen;
       }
     }
     return keyValueLength;
@@ -1013,7 +1017,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     int tagsLength = 0;
     if (tags != null && !tags.isEmpty()) {
       for (Tag t : tags) {
-        tagsLength += t.getLength();
+        tagsLength += t.getValueLength() + Tag.INFRASTRUCTURE_SIZE;
       }
     }
     checkForTagsLength(tagsLength);
@@ -1053,7 +1057,11 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tagsLength > 0) {
       pos = Bytes.putAsShort(bytes, pos, tagsLength);
       for (Tag t : tags) {
-        pos = Bytes.putBytes(bytes, pos, t.getBuffer(), t.getOffset(), t.getLength());
+        int tlen = t.getValueLength();
+        pos = Bytes.putAsShort(bytes, pos, tlen + Tag.TYPE_LENGTH_SIZE);
+        pos = Bytes.putByte(bytes, pos, t.getType());
+        TagUtil.copyValueTo(t, bytes, pos);
+        pos += tlen;
       }
     }
     return bytes;
@@ -1176,7 +1184,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tags != null) {
       List<String> tagsString = new ArrayList<String>();
       for (Tag t : tags) {
-        tagsString.add((t.getType()) + ":" +Bytes.toStringBinary(t.getValue()));
+        tagsString.add((t.getType()) + ":" + TagUtil.getValueAsString(t));
       }
       stringMap.put("tag", tagsString);
     }
@@ -1558,7 +1566,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tagsLength == 0) {
       return EMPTY_ARRAY_LIST;
     }
-    return Tag.asList(getTagsArray(), getTagsOffset(), tagsLength);
+    return TagUtil.asList(getTagsArray(), getTagsOffset(), tagsLength);
   }
 
   /**
@@ -2386,7 +2394,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
   public static KeyValue cloneAndAddTags(Cell c, List<Tag> newTags) {
     List<Tag> existingTags = null;
     if(c.getTagsLength() > 0) {
-      existingTags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+      existingTags = CellUtil.getTags(c);
       existingTags.addAll(newTags);
     } else {
       existingTags = newTags;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapTag.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapTag.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapTag.java
new file mode 100644
index 0000000..b3d65bb
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapTag.java
@@ -0,0 +1,83 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * This is a {@link Tag} implementation in which value is backed by an off heap
+ * {@link java.nio.ByteBuffer}
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OffheapTag implements Tag {
+
+  private ByteBuffer buffer;
+  private int offset, length;
+  private byte type;
+
+  public OffheapTag(ByteBuffer buffer, int offset, int length) {
+    this.buffer = buffer;
+    this.offset = offset;
+    this.length = length;
+    this.type = ByteBufferUtils.toByte(buffer, offset + TAG_LENGTH_SIZE);
+  }
+
+  @Override
+  public byte getType() {
+    return this.type;
+  }
+
+  @Override
+  public int getValueOffset() {
+    return this.offset + INFRASTRUCTURE_SIZE;
+  }
+
+  @Override
+  public int getValueLength() {
+    return this.length - INFRASTRUCTURE_SIZE;
+  }
+
+  @Override
+  public boolean hasArray() {
+    return false;
+  }
+
+  @Override
+  public byte[] getValueArray() {
+    throw new UnsupportedOperationException(
+        "Tag is backed by an off heap buffer. Use getValueByteBuffer()");
+  }
+
+  @Override
+  public ByteBuffer getValueByteBuffer() {
+    return this.buffer;
+  }
+
+  @Override
+  public String toString() {
+    return "[Tag type : " + this.type + ", value : "
+        + ByteBufferUtils.toStringBinary(buffer, getValueOffset(), getValueLength()) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
index 36b87b1..1d55baa 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
@@ -19,201 +19,60 @@
  */
 package org.apache.hadoop.hbase;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
+
 /**
- * Tags are part of cells and helps to add metadata about the KVs.
- * Metadata could be ACLs per cells, visibility labels, etc.
+ * Tags are part of cells and helps to add metadata about them.
+ * Metadata could be ACLs, visibility labels, etc.
+ * <p>
+ * Each Tag is having a type (one byte) and value part. The max value length for a Tag is 65533.
+ * <p>
+ * See {@link TagType} for reserved tag types.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class Tag {
+public interface Tag {
+
   public final static int TYPE_LENGTH_SIZE = Bytes.SIZEOF_BYTE;
   public final static int TAG_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
   public final static int INFRASTRUCTURE_SIZE = TYPE_LENGTH_SIZE + TAG_LENGTH_SIZE;
   public static final int MAX_TAG_LENGTH = (2 * Short.MAX_VALUE) + 1 - TAG_LENGTH_SIZE;
 
-  private final byte type;
-  private final byte[] bytes;
-  private int offset = 0;
-  private int length = 0;
-
-  /**
-   * The special tag will write the length of each tag and that will be
-   * followed by the type and then the actual tag.
-   * So every time the length part is parsed we need to add + 1 byte to it to
-   * get the type and then get the actual tag.
-   */
-  public Tag(byte tagType, String tag) {
-    this(tagType, Bytes.toBytes(tag));
-  }
-
-  /**
-   * Format for a tag :
-   * {@code <length of tag - 2 bytes><type code - 1 byte><tag>} tag length is serialized
-   * using 2 bytes only but as this will be unsigned, we can have max tag length of
-   * (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length.
-   */
-  public Tag(byte tagType, byte[] tag) {
-    int tagLength = tag.length + TYPE_LENGTH_SIZE;
-    if (tagLength > MAX_TAG_LENGTH) {
-      throw new IllegalArgumentException(
-          "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH);
-    }
-    length = TAG_LENGTH_SIZE + tagLength;
-    bytes = new byte[length];
-    int pos = Bytes.putAsShort(bytes, 0, tagLength);
-    pos = Bytes.putByte(bytes, pos, tagType);
-    Bytes.putBytes(bytes, pos, tag, 0, tag.length);
-    this.type = tagType;
-  }
-
-  /**
-   * Creates a Tag from the specified byte array and offset. Presumes
-   * <code>bytes</code> content starting at <code>offset</code> is formatted as
-   * a Tag blob.
-   * The bytes to include the tag type, tag length and actual tag bytes.
-   * @param offset offset to start of Tag
-   */
-  public Tag(byte[] bytes, int offset) {
-    this(bytes, offset, getLength(bytes, offset));
-  }
-
-  private static int getLength(byte[] bytes, int offset) {
-    return TAG_LENGTH_SIZE + Bytes.readAsInt(bytes, offset, TAG_LENGTH_SIZE);
-  }
-
-  /**
-   * Creates a Tag from the specified byte array, starting at offset, and for length
-   * <code>length</code>. Presumes <code>bytes</code> content starting at <code>offset</code> is
-   * formatted as a Tag blob.
-   */
-  public Tag(byte[] bytes, int offset, int length) {
-    if (length > MAX_TAG_LENGTH) {
-      throw new IllegalArgumentException(
-          "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH);
-    }
-    this.bytes = bytes;
-    this.offset = offset;
-    this.length = length;
-    this.type = bytes[offset + TAG_LENGTH_SIZE];
-  }
-
-  /**
-   * @return The byte array backing this Tag.
-   */
-  public byte[] getBuffer() {
-    return this.bytes;
-  }
-
   /**
    * @return the tag type
    */
-  public byte getType() {
-    return this.type;
-  }
-
-  /**
-   * @return Length of actual tag bytes within the backed buffer
-   */
-  public int getTagLength() {
-    return this.length - INFRASTRUCTURE_SIZE;
-  }
-
-  /**
-   * @return Offset of actual tag bytes within the backed buffer
-   */
-  public int getTagOffset() {
-    return this.offset + INFRASTRUCTURE_SIZE;
-  }
-
-  /**
-   * Returns tag value in a new byte array.
-   * Primarily for use client-side. If server-side, use
-   * {@link #getBuffer()} with appropriate {@link #getTagOffset()} and {@link #getTagLength()}
-   * instead to save on allocations.
-   * @return tag value in a new byte array.
-   */
-  public byte[] getValue() {
-    int tagLength = getTagLength();
-    byte[] tag = new byte[tagLength];
-    Bytes.putBytes(tag, 0, bytes, getTagOffset(), tagLength);
-    return tag;
-  }
+  byte getType();
 
   /**
-   * Creates the list of tags from the byte array b. Expected that b is in the
-   * expected tag format
-   * @param b
-   * @param offset
-   * @param length
-   * @return List of tags
+   * @return Offset of tag value within the backed buffer
    */
-  public static List<Tag> asList(byte[] b, int offset, int length) {
-    List<Tag> tags = new ArrayList<Tag>();
-    int pos = offset;
-    while (pos < offset + length) {
-      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
-      tags.add(new Tag(b, pos, tagLen + TAG_LENGTH_SIZE));
-      pos += TAG_LENGTH_SIZE + tagLen;
-    }
-    return tags;
-  }
+  int getValueOffset();
 
   /**
-   * Write a list of tags into a byte array
-   * @param tags
-   * @return the serialized tag data as bytes
+   * @return Length of tag value within the backed buffer
    */
-  public static byte[] fromList(List<Tag> tags) {
-    int length = 0;
-    for (Tag tag: tags) {
-      length += tag.length;
-    }
-    byte[] b = new byte[length];
-    int pos = 0;
-    for (Tag tag: tags) {
-      System.arraycopy(tag.bytes, tag.offset, b, pos, tag.length);
-      pos += tag.length;
-    }
-    return b;
-  }
+  int getValueLength();
 
   /**
-   * Retrieve the first tag from the tags byte array matching the passed in tag type
-   * @param b
-   * @param offset
-   * @param length
-   * @param type
-   * @return null if there is no tag of the passed in tag type
+   * Tells whether or not this Tag is backed by a byte array.
+   * @return true when this Tag is backed by byte array
    */
-  public static Tag getTag(byte[] b, int offset, int length, byte type) {
-    int pos = offset;
-    while (pos < offset + length) {
-      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
-      if(b[pos + TAG_LENGTH_SIZE] == type) {
-        return new Tag(b, pos, tagLen + TAG_LENGTH_SIZE);
-      }
-      pos += TAG_LENGTH_SIZE + tagLen;
-    }
-    return null;
-  }
+  boolean hasArray();
 
   /**
-   * Returns the total length of the entire tag entity
+   * @return The array containing the value bytes.
+   * @throws UnsupportedOperationException
+   *           when {@link #hasArray()} return false. Use {@link #getValueByteBuffer()} in such
+   *           situation
    */
-  int getLength() {
-    return this.length;
-  }
+  byte[] getValueArray();
 
   /**
-   * Returns the offset of the entire tag entity
+   * @return The {@link java.nio.ByteBuffer} containing the value bytes.
    */
-  int getOffset() {
-    return this.offset;
-  }
+  ByteBuffer getValueByteBuffer();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
new file mode 100644
index 0000000..15ddfc8
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
@@ -0,0 +1,219 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.util.StreamUtils;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+@InterfaceAudience.Private
+public final class TagUtil {
+
+  /**
+   * Private constructor to keep this class from being instantiated.
+   */
+  private TagUtil(){}
+
+  /**
+   * Returns tag value in a new byte array.
+   * Primarily for use client-side. If server-side, use
+   * {@link Tag#getValueArray()} with appropriate {@link Tag#getValueOffset()}
+   * and {@link Tag#getValueLength()} instead to save on allocations.
+   *
+   * @param tag The Tag whose value to be returned
+   * @return tag value in a new byte array.
+   */
+  public static byte[] cloneValue(Tag tag) {
+    int tagLength = tag.getValueLength();
+    byte[] tagArr = new byte[tagLength];
+    if (tag.hasArray()) {
+      Bytes.putBytes(tagArr, 0, tag.getValueArray(), tag.getValueOffset(), tagLength);
+    } else {
+      ByteBufferUtils.copyFromBufferToArray(tagArr, tag.getValueByteBuffer(), tag.getValueOffset(),
+          0, tagLength);
+    }
+    return tagArr;
+  }
+
+  /**
+   * Creates list of tags from given byte array, expected that it is in the expected tag format.
+   *
+   * @param b The byte array
+   * @param offset The offset in array where tag bytes begin
+   * @param length Total length of all tags bytes
+   * @return List of tags
+   */
+  public static List<Tag> asList(byte[] b, int offset, int length) {
+    List<Tag> tags = new ArrayList<Tag>();
+    int pos = offset;
+    while (pos < offset + length) {
+      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
+      tags.add(new ArrayBackedTag(b, pos, tagLen + TAG_LENGTH_SIZE));
+      pos += TAG_LENGTH_SIZE + tagLen;
+    }
+    return tags;
+  }
+
+  /**
+   * Creates list of tags from given ByteBuffer, expected that it is in the expected tag format.
+   *
+   * @param b The ByteBuffer
+   * @param offset The offset in ByteBuffer where tag bytes begin
+   * @param length Total length of all tags bytes
+   * @return List of tags
+   */
+  public static List<Tag> asList(ByteBuffer b, int offset, int length) {
+    List<Tag> tags = new ArrayList<Tag>();
+    int pos = offset;
+    while (pos < offset + length) {
+      int tagLen = ByteBufferUtils.readAsInt(b, pos, TAG_LENGTH_SIZE);
+      tags.add(new OffheapTag(b, pos, tagLen + TAG_LENGTH_SIZE));
+      pos += TAG_LENGTH_SIZE + tagLen;
+    }
+    return tags;
+  }
+
+  /**
+   * Write a list of tags into a byte array
+   *
+   * @param tags The list of tags
+   * @return the serialized tag data as bytes
+   */
+  public static byte[] fromList(List<Tag> tags) {
+    if (tags.isEmpty()) {
+      return HConstants.EMPTY_BYTE_ARRAY;
+    }
+    int length = 0;
+    for (Tag tag : tags) {
+      length += tag.getValueLength() + Tag.INFRASTRUCTURE_SIZE;
+    }
+    byte[] b = new byte[length];
+    int pos = 0;
+    int tlen;
+    for (Tag tag : tags) {
+      tlen = tag.getValueLength();
+      pos = Bytes.putAsShort(b, pos, tlen + Tag.TYPE_LENGTH_SIZE);
+      pos = Bytes.putByte(b, pos, tag.getType());
+      if (tag.hasArray()) {
+        pos = Bytes.putBytes(b, pos, tag.getValueArray(), tag.getValueOffset(), tlen);
+      } else {
+        ByteBufferUtils.copyFromBufferToArray(b, tag.getValueByteBuffer(), tag.getValueOffset(),
+            pos, tlen);
+        pos += tlen;
+      }
+    }
+    return b;
+  }
+
+  /**
+   * Converts the value bytes of the given tag into a long value
+   * @param tag The Tag
+   * @return value as long
+   */
+  public static long getValueAsLong(Tag tag) {
+    if (tag.hasArray()) {
+      return Bytes.toLong(tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
+    }
+    return ByteBufferUtils.toLong(tag.getValueByteBuffer(), tag.getValueOffset());
+  }
+
+  /**
+   * Converts the value bytes of the given tag into a byte value
+   * @param tag The Tag
+   * @return value as byte
+   */
+  public static byte getValueAsByte(Tag tag) {
+    if (tag.hasArray()) {
+      return tag.getValueArray()[tag.getValueOffset()];
+    }
+    return ByteBufferUtils.toByte(tag.getValueByteBuffer(), tag.getValueOffset());
+  }
+
+  /**
+   * Converts the value bytes of the given tag into a String value
+   * @param tag The Tag
+   * @return value as String
+   */
+  public static String getValueAsString(Tag tag){
+    if(tag.hasArray()){
+      return Bytes.toString(tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
+    }
+    return Bytes.toString(cloneValue(tag));
+  }
+
+  /**
+   * Matches the value part of given tags
+   * @param t1 Tag to match the value
+   * @param t2 Tag to match the value
+   * @return True if values of both tags are same.
+   */
+  public static boolean matchingValue(Tag t1, Tag t2) {
+    if (t1.hasArray() && t2.hasArray()) {
+      return Bytes.equals(t1.getValueArray(), t1.getValueOffset(), t1.getValueLength(),
+          t2.getValueArray(), t2.getValueOffset(), t2.getValueLength());
+    }
+    if (t1.hasArray()) {
+      return ByteBufferUtils.equals(t2.getValueByteBuffer(), t2.getValueOffset(),
+          t2.getValueLength(), t1.getValueArray(), t1.getValueOffset(), t1.getValueLength());
+    }
+    if (t2.hasArray()) {
+      return ByteBufferUtils.equals(t1.getValueByteBuffer(), t1.getValueOffset(),
+          t1.getValueLength(), t2.getValueArray(), t2.getValueOffset(), t2.getValueLength());
+    }
+    return ByteBufferUtils.equals(t1.getValueByteBuffer(), t1.getValueOffset(), t1.getValueLength(),
+        t2.getValueByteBuffer(), t2.getValueOffset(), t2.getValueLength());
+  }
+
+  /**
+   * Copies the tag's value bytes to the given byte array
+   * @param tag The Tag
+   * @param out The byte array where to copy the Tag value.
+   * @param offset The offset within 'out' array where to copy the Tag value.
+   */
+  public static void copyValueTo(Tag tag, byte[] out, int offset) {
+    if (tag.hasArray()) {
+      Bytes.putBytes(out, offset, tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
+    } else {
+      ByteBufferUtils.copyFromBufferToArray(out, tag.getValueByteBuffer(), tag.getValueOffset(),
+          offset, tag.getValueLength());
+    }
+  }
+
+  /**
+   * Reads an int value stored as a VInt at tag's given offset.
+   * @param tag The Tag
+   * @param offset The offset where VInt bytes begin
+   * @return A pair of the int value and number of bytes taken to store VInt
+   * @throws IOException When varint is malformed and not able to be read correctly
+   */
+  public static Pair<Integer, Integer> readVIntValuePart(Tag tag, int offset) throws IOException {
+    if (tag.hasArray()) {
+      return StreamUtils.readRawVarint32(tag.getValueArray(), offset);
+    }
+    return StreamUtils.readRawVarint32(tag.getValueByteBuffer(), offset);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
index 6e13b44..0e1c3ae 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.io.util;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.nio.ByteBuff;
@@ -127,9 +128,10 @@ public class StreamUtils {
    *          Offset in the input array where varInt is available
    * @return A pair of integers in which first value is the actual decoded varInt value and second
    *         value as number of bytes taken by this varInt for it's storage in the input array.
-   * @throws IOException
+   * @throws IOException When varint is malformed and not able to be read correctly
    */
-  public static Pair<Integer, Integer> readRawVarint32(byte[] input, int offset) throws IOException {
+  public static Pair<Integer, Integer> readRawVarint32(byte[] input, int offset)
+      throws IOException {
     int newOffset = offset;
     byte tmp = input[newOffset++];
     if (tmp >= 0) {
@@ -169,6 +171,47 @@ public class StreamUtils {
     return new Pair<Integer, Integer>(result, newOffset - offset);
   }
 
+  public static Pair<Integer, Integer> readRawVarint32(ByteBuffer input, int offset)
+      throws IOException {
+    int newOffset = offset;
+    byte tmp = input.get(newOffset++);
+    if (tmp >= 0) {
+      return new Pair<Integer, Integer>((int) tmp, newOffset - offset);
+    }
+    int result = tmp & 0x7f;
+    tmp = input.get(newOffset++);
+    if (tmp >= 0) {
+      result |= tmp << 7;
+    } else {
+      result |= (tmp & 0x7f) << 7;
+      tmp = input.get(newOffset++);
+      if (tmp >= 0) {
+        result |= tmp << 14;
+      } else {
+        result |= (tmp & 0x7f) << 14;
+        tmp = input.get(newOffset++);
+        if (tmp >= 0) {
+          result |= tmp << 21;
+        } else {
+          result |= (tmp & 0x7f) << 21;
+          tmp = input.get(newOffset++);
+          result |= tmp << 28;
+          if (tmp < 0) {
+            // Discard upper 32 bits.
+            for (int i = 0; i < 5; i++) {
+              tmp = input.get(newOffset++);
+              if (tmp >= 0) {
+                return new Pair<Integer, Integer>(result, newOffset - offset);
+              }
+            }
+            throw new IOException("Malformed varint");
+          }
+        }
+      }
+    }
+    return new Pair<Integer, Integer>(result, newOffset - offset);
+  }
+
   public static short toShort(byte hi, byte lo) {
     short s = (short) (((hi & 0xFF) << 8) | (lo & 0xFF));
     Preconditions.checkArgument(s >= 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 7bcc872..6e3fcaa 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -751,6 +751,29 @@ public final class ByteBufferUtils {
   }
 
   /**
+   * Converts a ByteBuffer to an int value
+   *
+   * @param buf The ByteBuffer
+   * @param offset Offset to int value
+   * @param length Number of bytes used to store the int value.
+   * @return the int value
+   * @throws IllegalArgumentException
+   *           if there's not enough bytes left in the buffer after the given offset
+   */
+  public static int readAsInt(ByteBuffer buf, int offset, final int length) {
+    if (offset + length > buf.limit()) {
+      throw new IllegalArgumentException("offset (" + offset + ") + length (" + length
+          + ") exceed the" + " limit of the buffer: " + buf.limit());
+    }
+    int n = 0;
+    for(int i = offset; i < (offset + length); i++) {
+      n <<= 8;
+      n ^= toByte(buf, i) & 0xFF;
+    }
+    return n;
+  }
+
+  /**
    * Reads a long value at the given buffer's offset.
    * @param buffer
    * @param offset

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
index b44a724..7dc3d5a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
@@ -280,8 +281,8 @@ public class RedundantKVGenerator {
       }
 
       if (useTags) {
-        result.add(new KeyValue(row, family, qualifier, timestamp, value, new Tag[] { new Tag(
-            (byte) 1, "value1") }));
+        result.add(new KeyValue(row, family, qualifier, timestamp, value,
+            new Tag[] { new ArrayBackedTag((byte) 1, "value1") }));
       } else {
         result.add(new KeyValue(row, family, qualifier, timestamp, value));
       }
@@ -365,7 +366,7 @@ public class RedundantKVGenerator {
       }
       if (useTags) {
         KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp, value,
-            new Tag[] { new Tag((byte) 1, "value1") });
+            new Tag[] { new ArrayBackedTag((byte) 1, "value1") });
         ByteBuffer offheapKVBB = ByteBuffer.allocateDirect(keyValue.getLength());
         ByteBufferUtils.copyFromArrayToBuffer(offheapKVBB, keyValue.getBuffer(),
           keyValue.getOffset(), keyValue.getLength());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
index cc1e511..e233348 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
@@ -442,7 +442,7 @@ public class TestKeyValue extends TestCase {
     byte[] metaValue1 = Bytes.toBytes("metaValue1");
     byte[] metaValue2 = Bytes.toBytes("metaValue2");
     KeyValue kv = new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, new Tag[] {
-        new Tag((byte) 1, metaValue1), new Tag((byte) 2, metaValue2) });
+        new ArrayBackedTag((byte) 1, metaValue1), new ArrayBackedTag((byte) 2, metaValue2) });
     assertTrue(kv.getTagsLength() > 0);
     assertTrue(Bytes.equals(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), row, 0,
       row.length));
@@ -458,44 +458,42 @@ public class TestKeyValue extends TestCase {
     boolean meta1Ok = false, meta2Ok = false;
     for (Tag tag : tags) {
       if (tag.getType() == (byte) 1) {
-        if (Bytes.equals(tag.getValue(), metaValue1)) {
+        if (Bytes.equals(TagUtil.cloneValue(tag), metaValue1)) {
           meta1Ok = true;
         }
       } else {
-        if (Bytes.equals(tag.getValue(), metaValue2)) {
+        if (Bytes.equals(TagUtil.cloneValue(tag), metaValue2)) {
           meta2Ok = true;
         }
       }
     }
     assertTrue(meta1Ok);
     assertTrue(meta2Ok);
-    Iterator<Tag> tagItr = CellUtil.tagsIterator(kv.getTagsArray(), kv.getTagsOffset(),
-        kv.getTagsLength());
+    Iterator<Tag> tagItr = CellUtil.tagsIterator(kv);
     //Iterator<Tag> tagItr = kv.tagsIterator();
     assertTrue(tagItr.hasNext());
     Tag next = tagItr.next();
-    assertEquals(10, next.getTagLength());
+    assertEquals(10, next.getValueLength());
     assertEquals((byte) 1, next.getType());
-    Bytes.equals(next.getValue(), metaValue1);
+    Bytes.equals(TagUtil.cloneValue(next), metaValue1);
     assertTrue(tagItr.hasNext());
     next = tagItr.next();
-    assertEquals(10, next.getTagLength());
+    assertEquals(10, next.getValueLength());
     assertEquals((byte) 2, next.getType());
-    Bytes.equals(next.getValue(), metaValue2);
+    Bytes.equals(TagUtil.cloneValue(next), metaValue2);
     assertFalse(tagItr.hasNext());
 
-    tagItr = CellUtil.tagsIterator(kv.getTagsArray(), kv.getTagsOffset(),
-        kv.getTagsLength());
+    tagItr = CellUtil.tagsIterator(kv);
     assertTrue(tagItr.hasNext());
     next = tagItr.next();
-    assertEquals(10, next.getTagLength());
+    assertEquals(10, next.getValueLength());
     assertEquals((byte) 1, next.getType());
-    Bytes.equals(next.getValue(), metaValue1);
+    Bytes.equals(TagUtil.cloneValue(next), metaValue1);
     assertTrue(tagItr.hasNext());
     next = tagItr.next();
-    assertEquals(10, next.getTagLength());
+    assertEquals(10, next.getValueLength());
     assertEquals((byte) 2, next.getType());
-    Bytes.equals(next.getValue(), metaValue2);
+    Bytes.equals(TagUtil.cloneValue(next), metaValue2);
     assertFalse(tagItr.hasNext());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
index f021215..9e76fc5 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
@@ -43,8 +43,8 @@ public class TestOffheapKeyValue {
   private static final byte[] fam2 = Bytes.toBytes(FAM2);
   private static final byte[] qual1 = Bytes.toBytes(QUAL1);
   private static final byte[] qual2 = Bytes.toBytes(QUAL2);
-  private static final Tag t1 = new Tag((byte) 1, Bytes.toBytes("TAG1"));
-  private static final Tag t2 = new Tag((byte) 2, Bytes.toBytes("TAG2"));
+  private static final Tag t1 = new ArrayBackedTag((byte) 1, Bytes.toBytes("TAG1"));
+  private static final Tag t2 = new ArrayBackedTag((byte) 2, Bytes.toBytes("TAG2"));
   private static final ArrayList<Tag> tags = new ArrayList<Tag>();
   static {
     tags.add(t1);
@@ -158,17 +158,17 @@ public class TestOffheapKeyValue {
     assertEquals(0L, offheapKV.getTimestamp());
     assertEquals(Type.Put.getCode(), offheapKV.getTypeByte());
     // change tags to handle both onheap and offheap stuff
-    List<Tag> resTags =
-        Tag.asList(offheapKV.getTagsArray(), offheapKV.getTagsOffset(), offheapKV.getTagsLength());
+    List<Tag> resTags = TagUtil.asList(offheapKV.getTagsArray(), offheapKV.getTagsOffset(),
+        offheapKV.getTagsLength());
     Tag tag1 = resTags.get(0);
     assertEquals(t1.getType(), tag1.getType());
-    assertEquals(Bytes.toString(t1.getValue()), Bytes.toString(getTagValue(tag1)));
+    assertEquals(TagUtil.getValueAsString(t1), TagUtil.getValueAsString(tag1));
     Tag tag2 = resTags.get(1);
     assertEquals(tag2.getType(), tag2.getType());
-    assertEquals(Bytes.toString(t2.getValue()), Bytes.toString(getTagValue(tag2)));
-    Tag res = Tag.getTag(offheapKV.getTagsArray(), 0, offheapKV.getTagsLength(), (byte) 2);
-    assertEquals(Bytes.toString(t2.getValue()), Bytes.toString(getTagValue(tag2)));
-    res = Tag.getTag(offheapKV.getTagsArray(), 0, offheapKV.getTagsLength(), (byte) 3);
+    assertEquals(TagUtil.getValueAsString(t2), TagUtil.getValueAsString(tag2));
+    Tag res = CellUtil.getTag(offheapKV, (byte) 2);
+    assertEquals(TagUtil.getValueAsString(t2), TagUtil.getValueAsString(tag2));
+    res = CellUtil.getTag(offheapKV, (byte) 3);
     assertNull(res);
   }
 
@@ -195,11 +195,4 @@ public class TestOffheapKeyValue {
     assertEquals(0L, offheapKeyOnlyKV.getTimestamp());
     assertEquals(Type.Put.getCode(), offheapKeyOnlyKV.getTypeByte());
   }
-  // TODO : Can be moved to TagUtil
-  private static byte[] getTagValue(Tag tag) {
-    int tagLength = tag.getTagLength();
-    byte[] tagBytes = new byte[tagLength];
-    System.arraycopy(tag.getBuffer(), tag.getTagOffset(), tagBytes, 0, tagLength);
-    return tagBytes;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
index beff87a..cc70742 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -54,16 +56,16 @@ public class TestCellCodecWithTags {
     Codec.Encoder encoder = codec.getEncoder(dos);
     final Cell cell1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"),
         HConstants.LATEST_TIMESTAMP, Bytes.toBytes("1"), new Tag[] {
-            new Tag((byte) 1, Bytes.toBytes("teststring1")),
-            new Tag((byte) 2, Bytes.toBytes("teststring2")) });
+            new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring1")),
+            new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring2")) });
     final Cell cell2 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("2"),
-        HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new Tag((byte) 1,
+        HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new ArrayBackedTag((byte) 1,
             Bytes.toBytes("teststring3")), });
     final Cell cell3 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("3"),
         HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"), new Tag[] {
-            new Tag((byte) 2, Bytes.toBytes("teststring4")),
-            new Tag((byte) 2, Bytes.toBytes("teststring5")),
-            new Tag((byte) 1, Bytes.toBytes("teststring6")) });
+            new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring4")),
+            new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring5")),
+            new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring6")) });
 
     encoder.write(cell1);
     encoder.write(cell2);
@@ -77,36 +79,36 @@ public class TestCellCodecWithTags {
     assertTrue(decoder.advance());
     Cell c = decoder.current();
     assertTrue(CellUtil.equals(c, cell1));
-    List<Tag> tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    List<Tag> tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
     assertEquals(2, tags.size());
     Tag tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), TagUtil.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), TagUtil.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, cell2));
-    tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
     assertEquals(1, tags.size());
     tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), TagUtil.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, cell3));
-    tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
     assertEquals(3, tags.size());
     tag = tags.get(0);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), TagUtil.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), TagUtil.cloneValue(tag)));
     tag = tags.get(2);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), TagUtil.cloneValue(tag)));
     assertFalse(decoder.advance());
     dis.close();
     assertEquals(offset, cis.getCount());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
index 04fb9a9..238d0a6 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -54,16 +56,16 @@ public class TestKeyValueCodecWithTags {
     Codec.Encoder encoder = codec.getEncoder(dos);
     final KeyValue kv1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"),
         HConstants.LATEST_TIMESTAMP, Bytes.toBytes("1"), new Tag[] {
-            new Tag((byte) 1, Bytes.toBytes("teststring1")),
-            new Tag((byte) 2, Bytes.toBytes("teststring2")) });
+            new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring1")),
+            new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring2")) });
     final KeyValue kv2 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("2"),
-        HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new Tag((byte) 1,
+        HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new ArrayBackedTag((byte) 1,
             Bytes.toBytes("teststring3")), });
     final KeyValue kv3 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("3"),
         HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"), new Tag[] {
-            new Tag((byte) 2, Bytes.toBytes("teststring4")),
-            new Tag((byte) 2, Bytes.toBytes("teststring5")),
-            new Tag((byte) 1, Bytes.toBytes("teststring6")) });
+            new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring4")),
+            new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring5")),
+            new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring6")) });
 
     encoder.write(kv1);
     encoder.write(kv2);
@@ -77,36 +79,36 @@ public class TestKeyValueCodecWithTags {
     assertTrue(decoder.advance());
     Cell c = decoder.current();
     assertTrue(CellUtil.equals(c, kv1));
-    List<Tag> tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    List<Tag> tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
     assertEquals(2, tags.size());
     Tag tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), TagUtil.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), TagUtil.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, kv2));
-    tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
     assertEquals(1, tags.size());
     tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), TagUtil.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, kv3));
-    tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
     assertEquals(3, tags.size());
     tag = tags.get(0);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), TagUtil.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), TagUtil.cloneValue(tag)));
     tag = tags.get(2);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), tag.getValue()));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), TagUtil.cloneValue(tag)));
     assertFalse(decoder.advance());
     dis.close();
     assertEquals(offset, cis.getCount());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
index f4c4afe..6c46cf2 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
@@ -28,6 +28,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -97,7 +98,7 @@ public class TestTagCompressionContext {
   private KeyValue createKVWithTags(int noOfTags) {
     List<Tag> tags = new ArrayList<Tag>();
     for (int i = 0; i < noOfTags; i++) {
-      tags.add(new Tag((byte) i, "tagValue" + i));
+      tags.add(new ArrayBackedTag((byte) i, "tagValue" + i));
     }
     KeyValue kv = new KeyValue(ROW, CF, Q, 1234L, V, tags);
     return kv;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java
index bd2a29d..717e24c 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Assert;
@@ -65,7 +66,7 @@ public class TestByteRangeWithKVSerialization {
     int kvCount = 1000000;
     List<KeyValue> kvs = new ArrayList<KeyValue>(kvCount);
     int totalSize = 0;
-    Tag[] tags = new Tag[] { new Tag((byte) 1, "tag1") };
+    Tag[] tags = new Tag[] { new ArrayBackedTag((byte) 1, "tag1") };
     for (int i = 0; i < kvCount; i++) {
       KeyValue kv = new KeyValue(Bytes.toBytes(i), FAMILY, QUALIFIER, i, VALUE, tags);
       kv.setSequenceId(i);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataTrivialWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataTrivialWithTags.java b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataTrivialWithTags.java
index 3c3699b..a615155 100644
--- a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataTrivialWithTags.java
+++ b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataTrivialWithTags.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
 import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
 import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
@@ -46,9 +47,9 @@ public class TestRowDataTrivialWithTags extends BaseTestRowData{
   static List<KeyValue> d = Lists.newArrayList();
   static {
     List<Tag> tagList = new ArrayList<Tag>();
-    Tag t = new Tag((byte) 1, "visisbility");
+    Tag t = new ArrayBackedTag((byte) 1, "visisbility");
     tagList.add(t);
-    t = new Tag((byte) 2, "ACL");
+    t = new ArrayBackedTag((byte) 2, "ACL");
     tagList.add(t);
     d.add(new KeyValue(rA, cf, cq0, ts, v0, tagList));
     d.add(new KeyValue(rB, cf, cq0, ts, v0, tagList));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
index 8424bf9..dcd5b0a 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.BufferedMutator;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -1124,7 +1125,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         byte[] tag = generateData(this.rand, TAG_LENGTH);
         Tag[] tags = new Tag[noOfTags];
         for (int n = 0; n < noOfTags; n++) {
-          Tag t = new Tag((byte) n, tag);
+          Tag t = new ArrayBackedTag((byte) n, tag);
           tags[n] = t;
         }
         KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP,
@@ -1195,7 +1196,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         byte[] tag = generateData(this.rand, TAG_LENGTH);
         Tag[] tags = new Tag[noOfTags];
         for (int n = 0; n < noOfTags; n++) {
-          Tag t = new Tag((byte) n, tag);
+          Tag t = new ArrayBackedTag((byte) n, tag);
           tags[n] = t;
         }
         KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index 86d183b..cc202d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -59,10 +59,11 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.mob.MobUtils;
@@ -367,11 +368,10 @@ public class HFilePrettyPrinter extends Configured implements Tool {
               + Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
                   cell.getValueLength()));
           int i = 0;
-          List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
+          List<Tag> tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(),
               cell.getTagsLength());
           for (Tag tag : tags) {
-            System.out.print(String.format(" T[%d]: %s", i++,
-                Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength())));
+            System.out.print(String.format(" T[%d]: %s", i++, TagUtil.getValueAsString(tag)));
           }
         }
         System.out.println();
@@ -411,7 +411,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
           System.err.println("ERROR, wrong value format in mob reference cell "
             + CellUtil.getCellKeyAsString(cell));
         } else {
-          TableName tn = TableName.valueOf(tnTag.getValue());
+          TableName tn = TableName.valueOf(TagUtil.cloneValue(tnTag));
           String mobFileName = MobUtils.getMobFileName(cell);
           boolean exist = mobFileExists(fs, tn, mobFileName,
             Bytes.toString(CellUtil.cloneFamily(cell)), foundMobFiles, missingMobFiles);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
index c201eb7..d2adbd4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java
@@ -25,6 +25,7 @@ import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
@@ -169,7 +170,7 @@ public class TextSortReducer extends
           // Add TTL directly to the KV so we can vary them when packing more than one KV
           // into puts
           if (ttl > 0) {
-            tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
+            tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
           }
           for (int i = 0; i < parsed.getColumnCount(); i++) {
             if (i == parser.getRowKeyColumnIndex() || i == parser.getTimestampKeyColumnIndex()

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
index 98dc25e..e14874b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
@@ -170,7 +171,7 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
         // Add TTL directly to the KV so we can vary them when packing more than one KV
         // into puts
         if (ttl > 0) {
-          tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
+          tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
         }
       }
       Put put = new Put(rowKey.copyBytes());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
index f48bb94..b5f412d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
@@ -167,7 +168,8 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
     byte[] fileName = null;
     StoreFile.Writer mobFileWriter = null, delFileWriter = null;
     long mobCells = 0, deleteMarkersCount = 0;
-    Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName().getName());
+    Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE,
+        store.getTableName().getName());
     long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
     long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
index ff350bf..999d25c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
@@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
@@ -166,8 +167,8 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
     // the relative path is mobFiles
     byte[] fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
     try {
-      Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName()
-          .getName());
+      Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE,
+          store.getTableName().getName());
       List<Cell> cells = new ArrayList<Cell>();
       boolean hasMore;
       ScannerContext scannerContext =

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
index 4bdfe97..82fc9cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.mob;
 
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
@@ -66,7 +67,7 @@ public final class MobConstants {
 
   public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period";
   public static final String MOB_CACHE_EVICT_REMAIN_RATIO = "hbase.mob.cache.evict.remain.ratio";
-  public static final Tag MOB_REF_TAG = new Tag(TagType.MOB_REFERENCE_TAG_TYPE,
+  public static final Tag MOB_REF_TAG = new ArrayBackedTag(TagType.MOB_REFERENCE_TAG_TYPE,
       HConstants.EMPTY_BYTE_ARRAY);
 
   public static final float DEFAULT_EVICT_REMAIN_RATIO = 0.5f;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index d654788..52a19f5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -122,8 +123,7 @@ public final class MobUtils {
    */
   public static boolean isMobReferenceCell(Cell cell) {
     if (cell.getTagsLength() > 0) {
-      Tag tag = Tag.getTag(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength(),
-          TagType.MOB_REFERENCE_TAG_TYPE);
+      Tag tag = CellUtil.getTag(cell, TagType.MOB_REFERENCE_TAG_TYPE);
       return tag != null;
     }
     return false;
@@ -136,9 +136,7 @@ public final class MobUtils {
    */
   public static Tag getTableNameTag(Cell cell) {
     if (cell.getTagsLength() > 0) {
-      Tag tag = Tag.getTag(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength(),
-          TagType.MOB_TABLE_NAME_TAG_TYPE);
-      return tag;
+      return CellUtil.getTag(cell, TagType.MOB_TABLE_NAME_TAG_TYPE);
     }
     return null;
   }
@@ -438,7 +436,7 @@ public final class MobUtils {
     // snapshot for mob files.
     tags.add(tableNameTag);
     // Add the existing tags.
-    tags.addAll(Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()));
+    tags.addAll(CellUtil.getTags(cell));
     int valueLength = cell.getValueLength();
     byte[] refValue = Bytes.add(Bytes.toBytes(valueLength), fileName);
     KeyValue reference = new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index ab9ee7e..6c6f115 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -113,7 +114,7 @@ public class PartitionedMobCompactor extends MobCompactor {
     Configuration copyOfConf = new Configuration(conf);
     copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
     compactionCacheConfig = new CacheConfig(copyOfConf);
-    tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
+    tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
     cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
index 3daef7e..5955cc2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
@@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -156,8 +157,8 @@ public class MemStoreWrapper {
     scanner = snapshot.getScanner();
     scanner.seek(KeyValueUtil.createFirstOnRow(HConstants.EMPTY_START_ROW));
     cell = null;
-    Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, Bytes.toBytes(this.table.getName()
-      .toString()));
+    Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE,
+        Bytes.toBytes(this.table.getName().toString()));
     long updatedCount = 0;
     while (null != (cell = scanner.next())) {
       KeyValue reference = MobUtils.createMobRefKeyValue(cell, referenceValue, tableNameTag);


[36/50] [abbrv] hbase git commit: HBASE-15027 Refactor the way the CompactedHFileDischarger threads are created (Ram)

Posted by jm...@apache.org.
HBASE-15027 Refactor the way the CompactedHFileDischarger threads are
created (Ram)


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

Branch: refs/heads/trunk
Commit: 28c2b18d30de4ce9564e328e5fdf42188e83fb63
Parents: f3ee6df
Author: ramkrishna <ra...@gmail.com>
Authored: Fri Jan 8 11:18:39 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Fri Jan 8 11:18:39 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/executor/EventType.java |   9 +-
 .../hadoop/hbase/executor/ExecutorType.java     |   3 +-
 .../hadoop/hbase/executor/ExecutorService.java  |   4 +-
 .../CompactedHFilesDischargeHandler.java        |  45 ++++++
 .../regionserver/CompactedHFilesDischarger.java | 112 ++++++++++++++
 .../hadoop/hbase/regionserver/HRegion.java      |  16 --
 .../hbase/regionserver/HRegionServer.java       |  26 +++-
 .../hadoop/hbase/regionserver/HStore.java       | 149 ++++++++-----------
 .../hbase/regionserver/OnlineRegions.java       |   6 +
 .../compactions/CompactedHFilesDischarger.java  |  74 ---------
 .../compactions/CompactionConfiguration.java    |   3 +
 .../hadoop/hbase/MockRegionServerServices.java  |   5 +
 .../TestZooKeeperTableArchiveClient.java        |  24 ++-
 .../hadoop/hbase/master/MockRegionServer.java   |   5 +
 .../master/cleaner/TestSnapshotFromMaster.java  |  19 ++-
 .../regionserver/TestHRegionReplayEvents.java   |  17 ++-
 .../TestRegionMergeTransactionOnCluster.java    |  36 +++--
 .../hbase/regionserver/TestRegionReplicas.java  |  15 +-
 .../TestCompactedHFilesDischarger.java          |  15 +-
 19 files changed, 368 insertions(+), 215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
index ac76edb..a7759c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
@@ -265,7 +265,14 @@ public enum EventType {
    *
    * RS_REGION_REPLICA_FLUSH
    */
-  RS_REGION_REPLICA_FLUSH   (82, ExecutorType.RS_REGION_REPLICA_FLUSH_OPS);
+  RS_REGION_REPLICA_FLUSH   (82, ExecutorType.RS_REGION_REPLICA_FLUSH_OPS),
+
+  /**
+   * RS compacted files discharger <br>
+   *
+   * RS_COMPACTED_FILES_DISCHARGER
+   */
+  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER);
 
   private final int code;
   private final ExecutorType executor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
index d0f6bee..5a16149 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
@@ -46,7 +46,8 @@ public enum ExecutorType {
   RS_CLOSE_META              (25),
   RS_PARALLEL_SEEK           (26),
   RS_LOG_REPLAY_OPS          (27),
-  RS_REGION_REPLICA_FLUSH_OPS  (28);
+  RS_REGION_REPLICA_FLUSH_OPS  (28),
+  RS_COMPACTED_FILES_DISCHARGER (29);
 
   ExecutorType(int value) {}
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
index 410fb39..335b672 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
@@ -37,6 +37,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.monitoring.ThreadMonitoring;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -78,7 +79,8 @@ public class ExecutorService {
    * started with the same name, this throws a RuntimeException.
    * @param name Name of the service to start.
    */
-  void startExecutorService(String name, int maxThreads) {
+  @VisibleForTesting
+  public void startExecutorService(String name, int maxThreads) {
     if (this.executorMap.get(name) != null) {
       throw new RuntimeException("An executor service with the name " + name +
         " is already running!");

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java
new file mode 100644
index 0000000..02160d8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischargeHandler.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.executor.EventHandler;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.HStore;
+
+/**
+ * Event handler that handles the removal and archival of the compacted hfiles
+ */
+@InterfaceAudience.Private
+public class CompactedHFilesDischargeHandler extends EventHandler {
+
+  private HStore store;
+
+  public CompactedHFilesDischargeHandler(Server server, EventType eventType, HStore store) {
+    super(server, eventType);
+    this.store = store;
+  }
+
+  @Override
+  public void process() throws IOException {
+    this.store.closeAndArchiveCompactedFiles();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
new file mode 100644
index 0000000..c4974cf
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.Store;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * A chore service that periodically cleans up the compacted files when there are no active readers
+ * using those compacted files and also helps in clearing the block cache with these compacted
+ * file entries
+ */
+@InterfaceAudience.Private
+public class CompactedHFilesDischarger extends ScheduledChore {
+  private static final Log LOG = LogFactory.getLog(CompactedHFilesDischarger.class);
+  private RegionServerServices regionServerServices;
+  // Default is to use executor
+  @VisibleForTesting
+  private boolean useExecutor = true;
+
+  /**
+   * @param period the period of time to sleep between each run
+   * @param stopper the stopper
+   * @param regionServerServices the region server that starts this chore
+   */
+  public CompactedHFilesDischarger(final int period, final Stoppable stopper,
+      final RegionServerServices regionServerServices) {
+    // Need to add the config classes
+    super("CompactedHFilesCleaner", stopper, period);
+    this.regionServerServices = regionServerServices;
+  }
+
+  /**
+   * @param period the period of time to sleep between each run
+   * @param stopper the stopper
+   * @param regionServerServices the region server that starts this chore
+   * @param useExecutor true if to use the region server's executor service, false otherwise
+   */
+  @VisibleForTesting
+  public CompactedHFilesDischarger(final int period, final Stoppable stopper,
+      final RegionServerServices regionServerServices, boolean useExecutor) {
+    // Need to add the config classes
+    this(period, stopper, regionServerServices);
+    this.useExecutor = useExecutor;
+  }
+
+  @Override
+  public void chore() {
+    List<Region> onlineRegions = regionServerServices.getOnlineRegions();
+    if (onlineRegions != null) {
+      for (Region region : onlineRegions) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(
+              "Started the compacted hfiles cleaner for the region " + region.getRegionInfo());
+        }
+        for (Store store : region.getStores()) {
+          try {
+            if (useExecutor && regionServerServices != null) {
+              CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
+                  (Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER,
+                  (HStore) store);
+              regionServerServices.getExecutorService().submit(handler);
+            } else {
+              // call synchronously if the RegionServerServices are not
+              // available
+              store.closeAndArchiveCompactedFiles();
+            }
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Completed archiving the compacted files for the region "
+                  + region.getRegionInfo() + " under the store " + store.getColumnFamilyName());
+            }
+          } catch (Exception e) {
+            LOG.error("Exception while trying to close and archive the comapcted store "
+                + "files of the store  " + store.getColumnFamilyName() + " in the" + " region "
+                + region.getRegionInfo(), e);
+          }
+        }
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(
+              "Completed the compacted hfiles cleaner for the region " + region.getRegionInfo());
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/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 38404c7..d059cd0 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
@@ -74,7 +74,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
@@ -151,7 +150,6 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescripto
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory;
@@ -814,20 +812,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // Initialize all the HStores
     status.setStatus("Initializing all the Stores");
     long maxSeqId = initializeStores(reporter, status);
-    // Start the CompactedHFilesDischarger here. This chore helps to remove the compacted files
-    // that will no longer be used in reads.
-    if (this.getRegionServerServices() != null) {
-      ChoreService choreService = this.getRegionServerServices().getChoreService();
-      if (choreService != null) {
-        // Default is 2 mins. The default value for TTLCleaner is 5 mins so we set this to
-        // 2 mins so that compacted files can be archived before the TTLCleaner runs
-        int cleanerInterval =
-            conf.getInt("hbase.hfile.compactions.cleaner.interval", 2 * 60 * 1000);
-        this.compactedFileDischarger =
-            new CompactedHFilesDischarger(cleanerInterval, this.getRegionServerServices(), this);
-        choreService.scheduleChore(compactedFileDischarger);
-      }
-    }
     this.mvcc.advanceTo(maxSeqId);
     if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
       // Recover any edits if available.

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/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 00046ba..b2cc78a 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
@@ -134,6 +134,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Repor
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
@@ -484,6 +485,8 @@ public class HRegionServer extends HasThread implements RegionServerServices, La
    */
   protected final ConfigurationManager configurationManager;
 
+  private CompactedHFilesDischarger compactedFileDischarger;
+
   /**
    * Starts a HRegionServer at the default location.
    * @param conf
@@ -615,6 +618,16 @@ public class HRegionServer extends HasThread implements RegionServerServices, La
         }
       });
     }
+    // Create the CompactedFileDischarger chore service. This chore helps to
+    // remove the compacted files
+    // that will no longer be used in reads.
+    // Default is 2 mins. The default value for TTLCleaner is 5 mins so we set this to
+    // 2 mins so that compacted files can be archived before the TTLCleaner runs
+    int cleanerInterval =
+        conf.getInt("hbase.hfile.compaction.discharger.interval", 2 * 60 * 1000);
+    this.compactedFileDischarger =
+        new CompactedHFilesDischarger(cleanerInterval, (Stoppable)this, (RegionServerServices)this);
+    choreService.scheduleChore(compactedFileDischarger);
   }
 
   protected TableDescriptors getFsTableDescriptors() throws IOException {
@@ -1716,7 +1729,9 @@ public class HRegionServer extends HasThread implements RegionServerServices, La
     }
     this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, conf.getInt(
        "hbase.regionserver.wal.max.splitters", SplitLogWorkerCoordination.DEFAULT_MAX_SPLITTERS));
-
+    // Start the threads for compacted files discharger
+    this.service.startExecutorService(ExecutorType.RS_COMPACTED_FILES_DISCHARGER,
+      conf.getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT, 10));
     if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) {
       this.service.startExecutorService(ExecutorType.RS_REGION_REPLICA_FLUSH_OPS,
         conf.getInt("hbase.regionserver.region.replica.flusher.threads",
@@ -2725,6 +2740,15 @@ public class HRegionServer extends HasThread implements RegionServerServices, La
      return tableRegions;
    }
 
+  @Override
+  public List<Region> getOnlineRegions() {
+    List<Region> allRegions = new ArrayList<Region>();
+    synchronized (this.onlineRegions) {
+      // Return a clone copy of the onlineRegions
+      allRegions.addAll(onlineRegions.values());
+    }
+    return allRegions;
+  }
   /**
    * Gets the online tables in this RS.
    * This method looks at the in-memory onlineRegions.

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 8d66696..9ebdaee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -37,9 +37,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -76,7 +74,6 @@ import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
@@ -91,7 +88,6 @@ import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
@@ -139,8 +135,6 @@ public class HStore implements Store {
   static int closeCheckInterval = 0;
   private volatile long storeSize = 0L;
   private volatile long totalUncompressedBytes = 0L;
-  private ThreadPoolExecutor compactionCleanerthreadPoolExecutor = null;
-  private CompletionService<StoreFile> completionService = null;
 
   /**
    * RWLock for store operations.
@@ -274,10 +268,6 @@ public class HStore implements Store {
           "hbase.hstore.flush.retries.number must be > 0, not "
               + flushRetriesNumber);
     }
-    compactionCleanerthreadPoolExecutor = getThreadPoolExecutor(
-      conf.getInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 10));
-    completionService =
-        new ExecutorCompletionService<StoreFile>(compactionCleanerthreadPoolExecutor);
     cryptoContext = EncryptionUtil.createEncryptionContext(conf, family);
   }
 
@@ -802,7 +792,9 @@ public class HStore implements Store {
       Collection<StoreFile> compactedfiles =
           storeEngine.getStoreFileManager().clearCompactedFiles();
       // clear the compacted files
-      removeCompactedFiles(compactedfiles);
+      if (compactedfiles != null && !compactedfiles.isEmpty()) {
+        removeCompactedfiles(compactedfiles);
+      }
       if (!result.isEmpty()) {
         // initialize the thread pool for closing store files in parallel.
         ThreadPoolExecutor storeFileCloserThreadPool = this.region
@@ -844,9 +836,6 @@ public class HStore implements Store {
         }
         if (ioe != null) throw ioe;
       }
-      if (compactionCleanerthreadPoolExecutor != null) {
-        compactionCleanerthreadPoolExecutor.shutdownNow();
-      }
       LOG.info("Closed " + this);
       return result;
     } finally {
@@ -2174,7 +2163,7 @@ public class HStore implements Store {
   }
 
   public static final long FIXED_OVERHEAD =
-      ClassSize.align(ClassSize.OBJECT + (18 * ClassSize.REFERENCE) + (10 * Bytes.SIZEOF_LONG)
+      ClassSize.align(ClassSize.OBJECT + (16 * ClassSize.REFERENCE) + (10 * Bytes.SIZEOF_LONG)
               + (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
 
   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
@@ -2311,92 +2300,72 @@ public class HStore implements Store {
     } finally {
       lock.readLock().unlock();
     }
-    removeCompactedFiles(copyCompactedfiles);
-  }
-
-  private ThreadPoolExecutor getThreadPoolExecutor(int maxThreads) {
-    return Threads.getBoundedCachedThreadPool(maxThreads, maxThreads * 3, TimeUnit.SECONDS,
-      new ThreadFactory() {
-        private int count = 1;
-
-        @Override
-        public Thread newThread(Runnable r) {
-          return new Thread(r, "CompactedfilesArchiver-" + count++);
-        }
-      });
+    if (copyCompactedfiles != null && !copyCompactedfiles.isEmpty()) {
+      removeCompactedfiles(copyCompactedfiles);
+    }
   }
 
-  private void removeCompactedFiles(Collection<StoreFile> compactedfiles) throws IOException {
-    if (compactedfiles != null && !compactedfiles.isEmpty()) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Removing the compacted store files " + compactedfiles);
-      }
-      for (final StoreFile file : compactedfiles) {
-        completionService.submit(new Callable<StoreFile>() {
-          @Override
-          public StoreFile call() throws IOException {
-            synchronized (file) {
-              try {
-                StoreFile.Reader r = file.getReader();
-                if (r == null) {
-                  if (LOG.isDebugEnabled()) {
-                    LOG.debug("The file " + file + " was closed but still not archived.");
-                  }
-                  return file;
-                }
-                if (r != null && r.isCompactedAway() && !r.isReferencedInReads()) {
-                  // Even if deleting fails we need not bother as any new scanners won't be
-                  // able to use the compacted file as the status is already compactedAway
-                  if (LOG.isTraceEnabled()) {
-                    LOG.trace("Closing and archiving the file " + file.getPath());
-                  }
-                  r.close(true);
-                  // Just close and return
-                  return file;
-                }
-              } catch (Exception e) {
-                LOG.error("Exception while trying to close the compacted store file "
-                    + file.getPath().getName());
-              }
+  /**
+   * Archives and removes the compacted files
+   * @param compactedfiles The compacted files in this store that are not active in reads
+   * @throws IOException
+   */
+  private void removeCompactedfiles(Collection<StoreFile> compactedfiles)
+      throws IOException {
+    final List<StoreFile> filesToRemove = new ArrayList<StoreFile>(compactedfiles.size());
+    for (final StoreFile file : compactedfiles) {
+      synchronized (file) {
+        try {
+          StoreFile.Reader r = file.getReader();
+          if (r == null) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("The file " + file + " was closed but still not archived.");
             }
-            return null;
+            filesToRemove.add(file);
           }
-        });
-      }
-      final List<StoreFile> filesToRemove = new ArrayList<StoreFile>(compactedfiles.size());
-      try {
-        for (final StoreFile file : compactedfiles) {
-          Future<StoreFile> future = completionService.take();
-          StoreFile closedFile = future.get();
-          if (closedFile != null) {
-            filesToRemove.add(closedFile);
+          if (r != null && r.isCompactedAway() && !r.isReferencedInReads()) {
+            // Even if deleting fails we need not bother as any new scanners won't be
+            // able to use the compacted file as the status is already compactedAway
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Closing and archiving the file " + file.getPath());
+            }
+            r.close(true);
+            // Just close and return
+            filesToRemove.add(file);
           }
+        } catch (Exception e) {
+          LOG.error(
+            "Exception while trying to close the compacted store file " + file.getPath().getName());
         }
-      } catch (InterruptedException ie) {
-        LOG.error("Interrupted exception while closing the compacted files", ie);
-      } catch (Exception e) {
-        LOG.error("Exception occured while closing the compacted files", e);
       }
-      if (isPrimaryReplicaStore()) {
-        archiveAndRemoveCompactedFiles(filesToRemove);
+    }
+    if (this.isPrimaryReplicaStore()) {
+      // Only the primary region is allowed to move the file to archive.
+      // The secondary region does not move the files to archive. Any active reads from
+      // the secondary region will still work because the file as such has active readers on it.
+      if (!filesToRemove.isEmpty()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Moving the files " + filesToRemove + " to archive");
+        }
+        // Only if this is successful it has to be removed
+        this.fs.removeStoreFiles(this.getFamily().getNameAsString(), filesToRemove);
       }
-
+    }
+    if (!filesToRemove.isEmpty()) {
+      // Clear the compactedfiles from the store file manager
+      clearCompactedfiles(filesToRemove);
     }
   }
 
-  private void archiveAndRemoveCompactedFiles(List<StoreFile> filesToArchive) throws IOException {
-    if (!filesToArchive.isEmpty()) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Moving the files " + filesToArchive + " to archive");
-      }
-      // Only if this is successful it has to be removed
-      this.fs.removeStoreFiles(this.getFamily().getNameAsString(), filesToArchive);
-      try {
-        lock.writeLock().lock();
-        this.getStoreEngine().getStoreFileManager().removeCompactedFiles(filesToArchive);
-      } finally {
-        lock.writeLock().unlock();
-      }
+  private void clearCompactedfiles(final List<StoreFile> filesToRemove) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Clearing the compacted file " + filesToRemove + " from this store");
+    }
+    try {
+      lock.writeLock().lock();
+      this.getStoreEngine().getStoreFileManager().removeCompactedFiles(filesToRemove);
+    } finally {
+      lock.writeLock().unlock();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
index 60fc9fb..310108c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
@@ -67,4 +67,10 @@ public interface OnlineRegions extends Server {
     * @throws java.io.IOException
     */
    List<Region> getOnlineRegions(TableName tableName) throws IOException;
+
+   /**
+    * Get all online regions in this RS.
+    * @return List of online Region
+    */
+   List<Region> getOnlineRegions();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactedHFilesDischarger.java
deleted file mode 100644
index 4cf120d..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactedHFilesDischarger.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver.compactions;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.Store;
-
-/**
- * A chore service that periodically cleans up the compacted files when there are no active readers
- * using those compacted files and also helps in clearing the block cache with these compacted
- * file entries
- */
-@InterfaceAudience.Private
-public class CompactedHFilesDischarger extends ScheduledChore {
-  private static final Log LOG = LogFactory.getLog(CompactedHFilesDischarger.class);
-  private Region region;
-
-  /**
-   * @param period the period of time to sleep between each run
-   * @param stopper the stopper
-   * @param region the store to identify the family name
-   */
-  public CompactedHFilesDischarger(final int period, final Stoppable stopper, final Region region) {
-    // Need to add the config classes
-    super("CompactedHFilesCleaner", stopper, period);
-    this.region = region;
-  }
-
-  @Override
-  public void chore() {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(
-        "Started the compacted hfiles cleaner for the region " + this.region.getRegionInfo());
-    }
-    for (Store store : region.getStores()) {
-      try {
-        store.closeAndArchiveCompactedFiles();
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Completed archiving the compacted files for the region "
-              + this.region.getRegionInfo() + " under the store " + store.getColumnFamilyName());
-        }
-      } catch (Exception e) {
-        LOG.error(
-          "Exception while trying to close and archive the comapcted store files of the store  "
-              + store.getColumnFamilyName() + " in the region " + this.region.getRegionInfo(),
-          e);
-      }
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(
-        "Completed the compacted hfiles cleaner for the region " + this.region.getRegionInfo());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
index 62e7c7c..633477e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
@@ -64,6 +64,9 @@ public class CompactionConfiguration {
   public static final String HBASE_HSTORE_MIN_LOCALITY_TO_SKIP_MAJOR_COMPACT =
       "hbase.hstore.min.locality.to.skip.major.compact";
 
+  public static final String HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT =
+      "hbase.hfile.compaction.discharger.thread.count";
+
   Configuration conf;
   StoreConfigInformation storeConfigInfo;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index a7fc75b..0986ad7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -110,6 +110,11 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
+  public List<Region> getOnlineRegions() {
+    return null;
+  }
+
+  @Override
   public void addToOnlineRegions(Region r) {
     this.regions.put(r.getRegionInfo().getEncodedName(), r);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index 55e43de..64139ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.backup.example;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -42,10 +44,11 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
+import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -79,6 +82,7 @@ public class TestZooKeeperTableArchiveClient {
   private static ZKTableArchiveClient archivingClient;
   private final List<Path> toCleanup = new ArrayList<Path>();
   private static ClusterConnection CONNECTION;
+  private static RegionServerServices rss;
 
   /**
    * Setup the config for the cluster
@@ -93,6 +97,7 @@ public class TestZooKeeperTableArchiveClient {
     ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
     String archivingZNode = ZKTableArchiveClient.getArchiveZNode(UTIL.getConfiguration(), watcher);
     ZKUtil.createWithParents(watcher, archivingZNode);
+    rss = mock(RegionServerServices.class);
   }
 
   private static void setupConf(Configuration conf) {
@@ -173,8 +178,11 @@ public class TestZooKeeperTableArchiveClient {
     // create the region
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
     HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
+    List<Region> regions = new ArrayList<Region>();
+    regions.add(region);
+    when(rss.getOnlineRegions()).thenReturn(regions);
     final CompactedHFilesDischarger compactionCleaner =
-        new CompactedHFilesDischarger(100, stop, region);
+        new CompactedHFilesDischarger(100, stop, rss, false);
     loadFlushAndCompact(region, TEST_FAM);
     compactionCleaner.chore();
     // get the current hfiles in the archive directory
@@ -223,15 +231,21 @@ public class TestZooKeeperTableArchiveClient {
     // create the region
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
     HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
+    List<Region> regions = new ArrayList<Region>();
+    regions.add(region);
+    when(rss.getOnlineRegions()).thenReturn(regions);
     final CompactedHFilesDischarger compactionCleaner =
-        new CompactedHFilesDischarger(100, stop, region);
+        new CompactedHFilesDischarger(100, stop, rss, false);
     loadFlushAndCompact(region, TEST_FAM);
     compactionCleaner.chore();
     // create the another table that we don't archive
     hcd = new HColumnDescriptor(TEST_FAM);
     HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd);
-    final CompactedHFilesDischarger compactionCleaner1 =
-        new CompactedHFilesDischarger(100, stop, otherRegion);
+    regions = new ArrayList<Region>();
+    regions.add(otherRegion);
+    when(rss.getOnlineRegions()).thenReturn(regions);
+    final CompactedHFilesDischarger compactionCleaner1 = new CompactedHFilesDischarger(100, stop,
+        rss, false);
     loadFlushAndCompact(otherRegion, TEST_FAM);
     compactionCleaner1.chore();
     // get the current hfiles in the archive directory

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 234ad20..32f644b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -462,6 +462,11 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
+  public List<Region> getOnlineRegions() {
+    return null;
+  }
+
+  @Override
   public OpenRegionResponse openRegion(RpcController controller,
       OpenRegionRequest request) throws ServiceException {
     // TODO Auto-generated method stub

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index 60c5473..a6b6e4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -46,11 +46,10 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnaps
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactedHFilesDischarger;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
@@ -60,6 +59,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -324,10 +324,17 @@ public class TestSnapshotFromMaster {
       region.waitForFlushesAndCompactions(); // enable can trigger a compaction, wait for it.
       region.compactStores(); // min is 2 so will compact and archive
     }
-    for (HRegion region : regions) {
-      CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, region);
-      cleaner.chore();
+    List<RegionServerThread> regionServerThreads = UTIL.getMiniHBaseCluster()
+        .getRegionServerThreads();
+    HRegionServer hrs = null;
+    for (RegionServerThread rs : regionServerThreads) {
+      if (!rs.getRegionServer().getOnlineRegions(TABLE_NAME).isEmpty()) {
+        hrs = rs.getRegionServer();
+        break;
+      }
     }
+    CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, hrs, false);
+    cleaner.chore();
     LOG.info("After compaction File-System state");
     FSUtils.logFileSystemState(fs, rootDir, LOG);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index c59d6f7..382193b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -69,7 +70,6 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescripto
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
 import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -167,9 +167,17 @@ public class TestHRegionReplayEvents {
     when(rss.getServerName()).thenReturn(ServerName.valueOf("foo", 1, 1));
     when(rss.getConfiguration()).thenReturn(CONF);
     when(rss.getRegionServerAccounting()).thenReturn(new RegionServerAccounting());
-
+    String string = org.apache.hadoop.hbase.executor.EventType.RS_COMPACTED_FILES_DISCHARGER
+        .toString();
+    ExecutorService es = new ExecutorService(string);
+    es.startExecutorService(
+      string+"-"+string, 1);
+    when(rss.getExecutorService()).thenReturn(es);
     primaryRegion = HRegion.createHRegion(primaryHri, rootDir, CONF, htd, walPrimary);
     primaryRegion.close();
+    List<Region> regions = new ArrayList<Region>();
+    regions.add(primaryRegion);
+    when(rss.getOnlineRegions()).thenReturn(regions);
 
     primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss, null);
     secondaryRegion = HRegion.openHRegion(secondaryHri, htd, null, CONF, rss, null);
@@ -1370,7 +1378,10 @@ public class TestHRegionReplayEvents {
 
     // Test case 3: compact primary files
     primaryRegion.compactStores();
-    CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, primaryRegion);
+    List<Region> regions = new ArrayList<Region>();
+    regions.add(primaryRegion);
+    when(rss.getOnlineRegions()).thenReturn(regions);
+    CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, rss, false);
     cleaner.chore();
     secondaryRegion.refreshStoreFiles();
     assertPathListsEqual(primaryRegion.getStoreFileList(families),

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index e0c1453..44b24ce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -62,12 +62,12 @@ import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.util.StringUtils;
@@ -246,25 +246,37 @@ public class TestRegionMergeTransactionOnCluster {
         count += hrfs.getStoreFiles(colFamily.getName()).size();
       }
       admin.compactRegion(mergedRegionInfo.getRegionName());
-      // wait until merged region doesn't have reference file
+      // clean up the merged region store files
+      // wait until merged region have reference file
       long timeout = System.currentTimeMillis() + waitTime;
+      int newcount = 0;
       while (System.currentTimeMillis() < timeout) {
-        if (!hrfs.hasReferences(tableDescriptor)) {
+        for(HColumnDescriptor colFamily : columnFamilies) {
+          newcount += hrfs.getStoreFiles(colFamily.getName()).size();
+        }
+        if(newcount > count) {
           break;
         }
         Thread.sleep(50);
       }
-      int newcount = 0;
-      for(HColumnDescriptor colFamily : columnFamilies) {
-        newcount += hrfs.getStoreFiles(colFamily.getName()).size();
-      }
       assertTrue(newcount > count);
-      // clean up the merged region store files
-      List<HRegion> regions = 
-          TEST_UTIL.getHBaseCluster().getRegions(tableDescriptor.getName());
-      for (HRegion region : regions) {
-        CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, region);
+      List<RegionServerThread> regionServerThreads = TEST_UTIL.getHBaseCluster()
+          .getRegionServerThreads();
+      for (RegionServerThread rs : regionServerThreads) {
+        CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null,
+            rs.getRegionServer(), false);
         cleaner.chore();
+        Thread.sleep(1000);
+      }
+      int newcount1 = 0;
+      while (System.currentTimeMillis() < timeout) {
+        for(HColumnDescriptor colFamily : columnFamilies) {
+          newcount1 += hrfs.getStoreFiles(colFamily.getName()).size();
+        }
+        if(newcount1 <= 1) {
+          break;
+        }
+        Thread.sleep(50);
       }
       // run CatalogJanitor to clean merge references in hbase:meta and archive the
       // files of merging regions

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
index 67258aa..99f5801 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.*;
 import java.io.IOException;
+import java.util.List;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -47,9 +48,9 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.AfterClass;
@@ -454,8 +455,18 @@ public class TestRegionReplicas {
       LOG.info("Force Major compaction on primary region " + hriPrimary);
       primaryRegion.compact(true);
       Assert.assertEquals(1, primaryRegion.getStore(f).getStorefilesCount());
+      List<RegionServerThread> regionServerThreads = HTU.getMiniHBaseCluster()
+          .getRegionServerThreads();
+      HRegionServer hrs = null;
+      for (RegionServerThread rs : regionServerThreads) {
+        if (rs.getRegionServer()
+            .getOnlineRegion(primaryRegion.getRegionInfo().getRegionName()) != null) {
+          hrs = rs.getRegionServer();
+          break;
+        }
+      }
       CompactedHFilesDischarger cleaner =
-          new CompactedHFilesDischarger(100, null, (HRegion) primaryRegion);
+          new CompactedHFilesDischarger(100, null, hrs, false);
       cleaner.chore();
       // scan all the hfiles on the secondary.
       // since there are no read on the secondary when we ask locations to

http://git-wip-us.apache.org/repos/asf/hbase/blob/28c2b18d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
index 40539c4..c23e794 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -38,10 +40,12 @@ import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -62,6 +66,7 @@ public class TestCompactedHFilesDischarger {
   private static CountDownLatch latch = new CountDownLatch(3);
   private static AtomicInteger counter = new AtomicInteger(0);
   private static AtomicInteger scanCompletedCounter = new AtomicInteger(0);
+  private RegionServerServices rss;
 
   @Before
   public void setUp() throws Exception {
@@ -71,6 +76,10 @@ public class TestCompactedHFilesDischarger {
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
     Path path = testUtil.getDataTestDir(getClass().getSimpleName());
     region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), htd);
+    rss = mock(RegionServerServices.class);
+    List<Region> regions = new ArrayList<Region>();
+    regions.add(region);
+    when(rss.getOnlineRegions()).thenReturn(regions);
   }
 
   @After
@@ -86,7 +95,7 @@ public class TestCompactedHFilesDischarger {
   public void testCompactedHFilesCleaner() throws Exception {
     // Create the cleaner object
     CompactedHFilesDischarger cleaner =
-        new CompactedHFilesDischarger(1000, (Stoppable) null, (HRegion) region);
+        new CompactedHFilesDischarger(1000, (Stoppable) null, rss, false);
     // Add some data to the region and do some flushes
     for (int i = 1; i < 10; i++) {
       Put p = new Put(Bytes.toBytes("row" + i));
@@ -152,7 +161,7 @@ public class TestCompactedHFilesDischarger {
   public void testCleanerWithParallelScannersAfterCompaction() throws Exception {
     // Create the cleaner object
     CompactedHFilesDischarger cleaner =
-        new CompactedHFilesDischarger(1000, (Stoppable) null, (HRegion) region);
+        new CompactedHFilesDischarger(1000, (Stoppable) null, rss, false);
     // Add some data to the region and do some flushes
     for (int i = 1; i < 10; i++) {
       Put p = new Put(Bytes.toBytes("row" + i));
@@ -223,7 +232,7 @@ public class TestCompactedHFilesDischarger {
   public void testCleanerWithParallelScanners() throws Exception {
     // Create the cleaner object
     CompactedHFilesDischarger cleaner =
-        new CompactedHFilesDischarger(1000, (Stoppable) null, (HRegion) region);
+        new CompactedHFilesDischarger(1000, (Stoppable) null, rss, false);
     // Add some data to the region and do some flushes
     for (int i = 1; i < 10; i++) {
       Put p = new Put(Bytes.toBytes("row" + i));


[03/50] [abbrv] hbase git commit: HBASE-14867 SimpleRegionNormalizer needs to have better heuristics to trigger merge operation

Posted by jm...@apache.org.
HBASE-14867 SimpleRegionNormalizer needs to have better heuristics to trigger merge operation


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

Branch: refs/heads/trunk
Commit: 1e4992c6eccb81166cdda842a68644fa962a3fdc
Parents: 6868c63
Author: tedyu <yu...@gmail.com>
Authored: Tue Dec 29 06:21:57 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Dec 29 06:21:57 2015 -0800

----------------------------------------------------------------------
 .../normalizer/SimpleRegionNormalizer.java      | 106 +++++++++----------
 .../normalizer/TestSimpleRegionNormalizer.java  |  43 +++++++-
 2 files changed, 90 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1e4992c6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index 659b3dc..fe6034b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -27,8 +27,11 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Triple;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 
 /**
@@ -66,6 +69,22 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
     this.masterServices = masterServices;
   }
 
+  /*
+   * This comparator compares the region size.
+   * The second element in the triple is region size while the 3rd element
+   * is the index of the region in the underlying List
+   */
+  private Comparator<Triple<HRegionInfo, Long, Integer>> regionSizeComparator =
+      new Comparator<Triple<HRegionInfo, Long, Integer>>() {
+    @Override
+    public int compare(Triple<HRegionInfo, Long, Integer> pair,
+        Triple<HRegionInfo, Long, Integer> pair2) {
+      long sz = pair.getSecond();
+      long sz2 = pair2.getSecond();
+      return (sz < sz2) ? -1 : ((sz == sz2) ? 0 : 1);
+    }
+  };
+
   /**
    * Computes next most "urgent" normalization action on the table.
    * Action may be either a split, or a merge, or no action.
@@ -76,7 +95,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
   @Override
   public NormalizationPlan computePlanForTable(TableName table) throws HBaseIOException {
     if (table == null || table.isSystemTable()) {
-      LOG.debug("Normalization of table " + table + " isn't allowed");
+      LOG.debug("Normalization of system table " + table + " isn't allowed");
       return EmptyNormalizationPlan.getInstance();
     }
 
@@ -95,57 +114,18 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
       ", number of regions: " + tableRegions.size());
 
     long totalSizeMb = 0;
-    Pair<HRegionInfo, Long> largestRegion = new Pair<>();
-
-    // A is a smallest region, B is it's smallest neighbor
-    Pair<HRegionInfo, Long> smallestRegion = new Pair<>();
-    int smallestRegionIndex = 0;
 
+    ArrayList<Triple<HRegionInfo, Long, Integer>> regionsWithSize =
+        new ArrayList<Triple<HRegionInfo, Long, Integer>>(tableRegions.size());
     for (int i = 0; i < tableRegions.size(); i++) {
       HRegionInfo hri = tableRegions.get(i);
       long regionSize = getRegionSize(hri);
+      regionsWithSize.add(new Triple<HRegionInfo, Long, Integer>(hri, regionSize, i));
       totalSizeMb += regionSize;
-
-      if (largestRegion.getFirst() == null || regionSize > largestRegion.getSecond()) {
-        largestRegion.setFirst(hri);
-        largestRegion.setSecond(regionSize);
-      }
-
-      if (smallestRegion.getFirst() == null || regionSize < smallestRegion.getSecond()) {
-        smallestRegion.setFirst(hri);
-        smallestRegion.setSecond(regionSize);
-        smallestRegionIndex = i;
-      }
     }
+    Collections.sort(regionsWithSize, regionSizeComparator);
 
-    // now get smallest neighbor of smallest region
-    long leftNeighborSize = -1;
-
-    if (smallestRegionIndex > 0) {
-      leftNeighborSize = getRegionSize(tableRegions.get(smallestRegionIndex - 1));
-    }
-
-    long rightNeighborSize = -1;
-    if (smallestRegionIndex < tableRegions.size() - 1) {
-      rightNeighborSize = getRegionSize(tableRegions.get(smallestRegionIndex + 1));
-    }
-
-    Pair<HRegionInfo, Long> smallestNeighborOfSmallestRegion;
-    if (leftNeighborSize == -1) {
-      smallestNeighborOfSmallestRegion =
-        new Pair<>(tableRegions.get(smallestRegionIndex + 1), rightNeighborSize);
-    } else if (rightNeighborSize == -1) {
-      smallestNeighborOfSmallestRegion =
-        new Pair<>(tableRegions.get(smallestRegionIndex - 1), leftNeighborSize);
-    } else {
-      if (leftNeighborSize < rightNeighborSize) {
-        smallestNeighborOfSmallestRegion =
-          new Pair<>(tableRegions.get(smallestRegionIndex - 1), leftNeighborSize);
-      } else {
-        smallestNeighborOfSmallestRegion =
-          new Pair<>(tableRegions.get(smallestRegionIndex + 1), rightNeighborSize);
-      }
-    }
+    Triple<HRegionInfo, Long, Integer> largestRegion = regionsWithSize.get(tableRegions.size()-1);
 
     double avgRegionSize = totalSizeMb / (double) tableRegions.size();
 
@@ -159,19 +139,31 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
         + largestRegion.getFirst().getRegionNameAsString() + " has size "
         + largestRegion.getSecond() + ", more than 2 times than avg size, splitting");
       return new SplitNormalizationPlan(largestRegion.getFirst(), null);
-    } else {
-      if (smallestRegion.getSecond() + smallestNeighborOfSmallestRegion.getSecond()
-          < avgRegionSize) {
-        LOG.debug("Table " + table + ", smallest region size: " + smallestRegion.getSecond()
-          + " and its smallest neighbor size: " + smallestNeighborOfSmallestRegion.getSecond()
-          + ", less than the avg size, merging them");
-        return new MergeNormalizationPlan(smallestRegion.getFirst(),
-          smallestNeighborOfSmallestRegion.getFirst());
-      } else {
-        LOG.debug("No normalization needed, regions look good for table: " + table);
-        return EmptyNormalizationPlan.getInstance();
+    }
+    int candidateIdx = 0;
+    // look for two successive entries whose indices are adjacent
+    while (candidateIdx < tableRegions.size()-1) {
+      if (Math.abs(regionsWithSize.get(candidateIdx).getThird() -
+        regionsWithSize.get(candidateIdx + 1).getThird()) == 1) {
+        break;
       }
+      candidateIdx++;
+    }
+    if (candidateIdx == tableRegions.size()-1) {
+      LOG.debug("No neighboring regions found for table: " + table);
+      return EmptyNormalizationPlan.getInstance();
+    }
+    Triple<HRegionInfo, Long, Integer> candidateRegion = regionsWithSize.get(candidateIdx);
+    Triple<HRegionInfo, Long, Integer> candidateRegion2 = regionsWithSize.get(candidateIdx+1);
+    if (candidateRegion.getSecond() + candidateRegion2.getSecond() < avgRegionSize) {
+      LOG.debug("Table " + table + ", smallest region size: " + candidateRegion.getSecond()
+        + " and its smallest neighbor size: " + candidateRegion2.getSecond()
+        + ", less than the avg size, merging them");
+      return new MergeNormalizationPlan(candidateRegion.getFirst(),
+        candidateRegion2.getFirst());
     }
+    LOG.debug("No normalization needed, regions look good for table: " + table);
+    return EmptyNormalizationPlan.getInstance();
   }
 
   private long getRegionSize(HRegionInfo hri) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1e4992c6/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
index 211911e..970af43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
@@ -142,8 +142,8 @@ public class TestSimpleRegionNormalizer {
     hris.add(hri4);
     regionSizes.put(hri4.getRegionName(), 15);
 
-    HRegionInfo hri5 = new HRegionInfo(testTable, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
-    hris.add(hri4);
+    HRegionInfo hri5 = new HRegionInfo(testTable, Bytes.toBytes("eee"), Bytes.toBytes("fff"));
+    hris.add(hri5);
     regionSizes.put(hri5.getRegionName(), 16);
 
     setupMocksForNormalizer(regionSizes, hris);
@@ -154,6 +154,45 @@ public class TestSimpleRegionNormalizer {
     assertEquals(hri3, ((MergeNormalizationPlan) plan).getSecondRegion());
   }
 
+  // Test for situation illustrated in HBASE-14867
+  @Test
+  public void testMergeOfSecondSmallestRegions() throws HBaseIOException {
+    TableName testTable = TableName.valueOf("testMergeOfSmallRegions");
+    List<HRegionInfo> hris = new ArrayList<>();
+    Map<byte[], Integer> regionSizes = new HashMap<>();
+
+    HRegionInfo hri1 = new HRegionInfo(testTable, Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
+    hris.add(hri1);
+    regionSizes.put(hri1.getRegionName(), 1);
+
+    HRegionInfo hri2 = new HRegionInfo(testTable, Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
+    hris.add(hri2);
+    regionSizes.put(hri2.getRegionName(), 10000);
+
+    HRegionInfo hri3 = new HRegionInfo(testTable, Bytes.toBytes("ccc"), Bytes.toBytes("ddd"));
+    hris.add(hri3);
+    regionSizes.put(hri3.getRegionName(), 10000);
+
+    HRegionInfo hri4 = new HRegionInfo(testTable, Bytes.toBytes("ddd"), Bytes.toBytes("eee"));
+    hris.add(hri4);
+    regionSizes.put(hri4.getRegionName(), 10000);
+
+    HRegionInfo hri5 = new HRegionInfo(testTable, Bytes.toBytes("eee"), Bytes.toBytes("fff"));
+    hris.add(hri5);
+    regionSizes.put(hri5.getRegionName(), 2700);
+
+    HRegionInfo hri6 = new HRegionInfo(testTable, Bytes.toBytes("fff"), Bytes.toBytes("ggg"));
+    hris.add(hri6);
+    regionSizes.put(hri6.getRegionName(), 2700);
+
+    setupMocksForNormalizer(regionSizes, hris);
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
+
+    assertTrue(plan instanceof MergeNormalizationPlan);
+    assertEquals(hri5, ((MergeNormalizationPlan) plan).getFirstRegion());
+    assertEquals(hri6, ((MergeNormalizationPlan) plan).getSecondRegion());
+  }
+
   @Test
   public void testMergeOfSmallNonAdjacentRegions() throws HBaseIOException {
     TableName testTable = TableName.valueOf("testMergeOfSmallRegions");


[06/50] [abbrv] hbase git commit: HBASE-15007 update docs to list Hadoop 2.6.1+ as cool.

Posted by jm...@apache.org.
HBASE-15007 update docs to list Hadoop 2.6.1+ as cool.


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

Branch: refs/heads/trunk
Commit: 5e2c2e1780d57f2e0086b10cd28d5212d6a3d5d0
Parents: 822fead
Author: Sean Busbey <bu...@apache.org>
Authored: Tue Dec 29 11:46:17 2015 -0600
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Dec 29 11:46:17 2015 -0600

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/configuration.adoc | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5e2c2e17/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index e7deea3..495232f 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -222,7 +222,8 @@ Use the following legend to interpret this table:
 |Hadoop-2.3.x | NT | S | NT | NT | NT
 |Hadoop-2.4.x | NT | S | S | S | S
 |Hadoop-2.5.x | NT | S | S | S | S
-|Hadoop-2.6.x | X | X | X | X | X
+|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
 |===
@@ -233,7 +234,7 @@ Use the following legend to interpret this table:
 Hadoop distributions based on the 2.6.x line *must* have
 link:https://issues.apache.org/jira/browse/HADOOP-11710[HADOOP-11710] applied if you plan to run
 HBase on top of an HDFS Encryption Zone. Failure to do so will result in cluster failure and
-data loss.
+data loss. This patch is present in Apache Hadoop releases 2.6.1+.
 ====
 
 .Hadoop 2.7.x


[22/50] [abbrv] hbase git commit: HBASE-14888 ClusterSchema: Add Namespace Operations

Posted by jm...@apache.org.
HBASE-14888 ClusterSchema: Add Namespace Operations


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

Branch: refs/heads/trunk
Commit: 46303dfd751912371539aa41ee8698dfb5e8b304
Parents: 1c4edd2
Author: stack <st...@apache.org>
Authored: Tue Jan 5 14:35:27 2016 -0800
Committer: stack <st...@apache.org>
Committed: Tue Jan 5 14:35:27 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  50 +-
 .../hadoop/hbase/client/ClusterConnection.java  |   1 +
 .../hbase/client/ConnectionImplementation.java  |  55 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 231 +++---
 .../coprocessor/BypassCoprocessorException.java |  44 +
 .../java/org/apache/hadoop/hbase/Service.java   |  50 ++
 .../hbase/ServiceNotRunningException.java       |  39 +
 .../apache/hadoop/hbase/ResourceChecker.java    |   2 +-
 .../hbase/protobuf/generated/MasterProtos.java  | 794 +++++++++++++------
 hbase-protocol/src/main/protobuf/Master.proto   |   3 +
 .../java/org/apache/hadoop/hbase/Server.java    |  13 +-
 .../hadoop/hbase/master/ClusterSchema.java      | 131 +++
 .../hbase/master/ClusterSchemaException.java    |  37 +
 .../hbase/master/ClusterSchemaService.java      |  27 +
 .../hbase/master/ClusterSchemaServiceImpl.java  | 131 +++
 .../org/apache/hadoop/hbase/master/HMaster.java | 414 ++++------
 .../hadoop/hbase/master/MasterRpcServices.java  | 139 +---
 .../hadoop/hbase/master/MasterServices.java     |  78 +-
 .../hadoop/hbase/master/ServerManager.java      |   2 +-
 .../hbase/master/TableNamespaceManager.java     |  99 ++-
 .../procedure/CreateNamespaceProcedure.java     |   2 +-
 .../procedure/DeleteNamespaceProcedure.java     |   2 +-
 .../master/procedure/DeleteTableProcedure.java  |   4 +-
 .../procedure/ModifyNamespaceProcedure.java     |   3 +-
 .../master/procedure/ServerCrashProcedure.java  |   2 +-
 .../hbase/namespace/NamespaceStateManager.java  |   6 +-
 .../hbase/regionserver/HRegionServer.java       |  11 +-
 .../regionserver/ReplicationSyncUp.java         |   6 +
 .../resources/hbase-webapps/master/table.jsp    |   4 +-
 .../hadoop/hbase/MockRegionServerServices.java  |   6 +
 .../org/apache/hadoop/hbase/TestNamespace.java  |   2 +-
 .../client/TestShortCircuitConnection.java      |   2 +-
 .../hbase/coprocessor/TestMasterObserver.java   |  24 +-
 .../hadoop/hbase/master/MockRegionServer.java   |   6 +
 .../hbase/master/TestActiveMasterManager.java   |   6 +
 .../hadoop/hbase/master/TestCatalogJanitor.java |  73 +-
 .../hbase/master/TestClockSkewDetection.java    |   6 +
 .../hbase/master/TestMasterNoCluster.java       |  23 +-
 .../hbase/master/TestSplitLogManager.java       |   6 +
 .../balancer/TestRegionLocationFinder.java      |   5 +-
 .../hbase/master/cleaner/TestHFileCleaner.java  |   6 +
 .../master/cleaner/TestHFileLinkCleaner.java    |   6 +
 .../hbase/master/cleaner/TestLogsCleaner.java   |   6 +
 .../cleaner/TestReplicationHFileCleaner.java    |   6 +
 .../MasterProcedureTestingUtility.java          |   4 +-
 .../regionserver/TestHeapMemoryManager.java     |   6 +
 .../hbase/regionserver/TestSplitLogWorker.java  |   6 +
 .../replication/TestReplicationStateZKImpl.java |   6 +
 .../TestReplicationTrackerZKImpl.java           |   7 +
 .../TestReplicationSourceManager.java           |   6 +
 .../security/token/TestTokenAuthentication.java |   6 +
 .../apache/hadoop/hbase/util/MockServer.java    |   6 +
 52 files changed, 1615 insertions(+), 995 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index b06902a..d7b52d5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -965,49 +965,77 @@ public interface Admin extends Abortable, Closeable {
   Configuration getConfiguration();
 
   /**
-   * Create a new namespace
+   * Create a new namespace. Blocks until namespace has been successfully created or an exception
+   * is thrown.
    *
    * @param descriptor descriptor which describes the new namespace
-   * @throws IOException
    */
   void createNamespace(final NamespaceDescriptor descriptor)
-      throws IOException;
+  throws IOException;
 
   /**
-   * Modify an existing namespace
+   * Create a new namespace
+   *
+   * @param descriptor descriptor which describes the new namespace
+   * @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to
+   *  wait on the operation to complete.
+   */
+  Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
+  throws IOException;
+
+  /**
+   * Modify an existing namespace.  Blocks until namespace has been successfully modified or an
+   * exception is thrown.
    *
    * @param descriptor descriptor which describes the new namespace
-   * @throws IOException
    */
   void modifyNamespace(final NamespaceDescriptor descriptor)
-      throws IOException;
+  throws IOException;
+
+  /**
+   * Modify an existing namespace
+   *
+   * @param descriptor descriptor which describes the new namespace
+   * @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to
+   *  wait on the operation to complete.
+   */
+  Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
+  throws IOException;
 
   /**
    * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+   * Blocks until namespace has been successfully deleted or an
+   * exception is thrown.
    *
    * @param name namespace name
-   * @throws IOException
    */
   void deleteNamespace(final String name) throws IOException;
 
   /**
+   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+   *
+   * @param name namespace name
+   * @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to
+   *  wait on the operation to complete.
+   */
+  Future<Void> deleteNamespaceAsync(final String name) throws IOException;
+
+  /**
    * Get a namespace descriptor by name
    *
    * @param name name of namespace descriptor
    * @return A descriptor
-   * @throws IOException
    */
   NamespaceDescriptor getNamespaceDescriptor(final String name)
-      throws IOException;
+  throws IOException;
 
   /**
    * List available namespace descriptors
    *
    * @return List of descriptors
-   * @throws IOException
    */
   NamespaceDescriptor[] listNamespaceDescriptors()
-    throws IOException;
+  throws IOException;
 
   /**
    * Get list of table descriptors by namespace

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
index 99071fa..741989f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
@@ -1,5 +1,6 @@
 /**
  *
+
  * 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 0ef2a17..ecac792 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -150,8 +150,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   //  be waiting for the master lock => deadlock.
   private final Object masterAndZKLock = new Object();
 
-  private long keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
-
   // thread executor shared by all HTableInterface instances created
   // by this connection
   private volatile ExecutorService batchPool = null;
@@ -398,7 +396,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       synchronized (this) {
         if (batchPool == null) {
           this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256),
-              conf.getInt("hbase.hconnection.threads.core", 256), "-shared-", null);
+              conf.getInt("hbase.hconnection.threads.core", 256), "-shared", null);
           this.cleanupPool = true;
         }
       }
@@ -482,7 +480,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
 
   /**
    * @return The cluster registry implementation to use.
-   * @throws java.io.IOException
    */
   private Registry setupRegistry() throws IOException {
     return RegistryFactory.getRegistry(this);
@@ -542,7 +539,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   /**
    * @return true if the master is running, throws an exception otherwise
    * @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running
-   * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
    * @deprecated this has been deprecated without a replacement
    */
   @Deprecated
@@ -981,9 +977,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   /**
    * Search the cache for a location that fits our table and row key.
    * Return null if no suitable region is located.
-   *
-   * @param tableName
-   * @param row
    * @return Null or region location found in cache.
    */
   RegionLocations getCachedLocation(final TableName tableName,
@@ -1181,13 +1174,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
 
     /**
      * Make stub and cache it internal so can be used later doing the isMasterRunning call.
-     * @param channel
      */
     protected abstract Object makeStub(final BlockingRpcChannel channel);
 
     /**
      * Once setup, check it works by doing isMasterRunning check.
-     * @throws com.google.protobuf.ServiceException
      */
     protected abstract void isMasterRunning() throws ServiceException;
 
@@ -1195,9 +1186,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
      * Create a stub. Try once only.  It is not typed because there is no common type to
      * protobuf services nor their interfaces.  Let the caller do appropriate casting.
      * @return A stub for master services.
-     * @throws java.io.IOException
-     * @throws org.apache.zookeeper.KeeperException
-     * @throws com.google.protobuf.ServiceException
      */
     private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException {
       ZooKeeperKeepAliveConnection zkw;
@@ -1370,10 +1358,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
 
   private ZooKeeperKeepAliveConnection keepAliveZookeeper;
   private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0);
-  private boolean canCloseZKW = true;
-
-  // keepAlive time, in ms. No reason to make it configurable.
-  private static final long keepAlive = 5 * 60 * 1000;
 
   /**
    * Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it.
@@ -1391,7 +1375,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
         keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this);
       }
       keepAliveZookeeperUserCount.addAndGet(1);
-      keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
       return keepAliveZookeeper;
     }
   }
@@ -1400,9 +1383,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     if (zkw == null){
       return;
     }
-    if (keepAliveZookeeperUserCount.addAndGet(-1) <= 0) {
-      keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive;
-    }
   }
 
   private void closeZooKeeperWatcher() {
@@ -1820,7 +1800,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     };
   }
 
-
   private static void release(MasterServiceState mss) {
     if (mss != null && mss.connection != null) {
       ((ConnectionImplementation)mss.connection).releaseMaster(mss);
@@ -1893,7 +1872,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    * or delete it from the cache. Does nothing if we can be sure from the exception that
    * the location is still accurate, or if the cache has already been updated.
    * @param exception an object (to simplify user code) on which we will try to find a nested
-   *                  or wrapped or both RegionMovedException
+   *  or wrapped or both RegionMovedException
    * @param source server that is the source of the location update.
    */
   @Override
@@ -1964,7 +1943,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   /**
-   * @deprecated since 0.96 - Use {@link org.apache.hadoop.hbase.client.HTableInterface#batch} instead
+   * @deprecated since 0.96 Use {@link org.apache.hadoop.hbase.client.HTableInterface#batch} instead
    */
   @Override
   @Deprecated
@@ -1999,8 +1978,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    * If the method returns it means that there is no error, and the 'results' array will
    * contain no exception. On error, an exception is thrown, and the 'results' array will
    * contain results and exceptions.
-   * @deprecated since 0.96 -
-   *   Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead
+   * @deprecated since 0.96
+   *  Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead
    */
   @Override
   @Deprecated
@@ -2225,7 +2204,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   /**
-   * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead
+   * @deprecated Use {@link
+   *  org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead
    */
   @Deprecated
   @Override
@@ -2245,12 +2225,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   /**
-   * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead
+   * @deprecated Use
+   *  {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)}
+   *  instead
    */
   @Deprecated
   @Override
-  public HTableDescriptor[] getHTableDescriptors(
-      List<String> names) throws IOException {
+  public HTableDescriptor[] getHTableDescriptors(List<String> names) throws IOException {
     List<TableName> tableNames = new ArrayList<TableName>(names.size());
     for(String name : names) {
       tableNames.add(TableName.valueOf(name));
@@ -2269,7 +2250,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
    * @param tableName table name
    * @throws java.io.IOException if the connection to master fails or if the table
    *  is not found.
-   * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)} instead
+   * @deprecated Use {@link
+   *  org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)}
+   *  instead
    */
   @Deprecated
   @Override
@@ -2294,7 +2277,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   }
 
   /**
-   * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)} instead
+   * @deprecated Use {@link
+   *  org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)}
+   *  instead
    */
   @Deprecated
   @Override
@@ -2306,10 +2291,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   @Override
   public TableState getTableState(TableName tableName) throws IOException {
     if (this.closed) throw new IOException(toString() + " closed");
-
     TableState tableState = MetaTableAccessor.getTableState(this, tableName);
-    if (tableState == null)
-      throw new TableNotFoundException(tableName);
+    if (tableState == null) throw new TableNotFoundException(tableName);
     return tableState;
   }
 
@@ -2318,4 +2301,4 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return RpcRetryingCallerFactory
         .instantiate(conf, this.interceptor, this.getStatisticsTracker());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 7a50458..db94ff4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -28,12 +28,12 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
@@ -95,11 +95,13 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
@@ -135,6 +137,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTi
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
@@ -254,23 +257,10 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public boolean abortProcedure(
-      final long procId,
-      final boolean mayInterruptIfRunning) throws IOException {
-    Future<Boolean> future = abortProcedureAsync(procId, mayInterruptIfRunning);
-    try {
-      return future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException("Interrupted when waiting for procedure to be cancelled");
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException)e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
-    }
+  public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
+  throws IOException {
+    return get(abortProcedureAsync(procId, mayInterruptIfRunning), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
   }
 
   @Override
@@ -464,22 +454,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
       throws IOException {
-    Future<Void> future = createTableAsync(desc, splitKeys);
-    try {
-      // TODO: how long should we wait? spin forever?
-      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException("Interrupted when waiting" +
-          " for table to be enabled; meta scan was done");
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException)e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
-    }
+    get(createTableAsync(desc, splitKeys), syncWaitTimeout, TimeUnit.MILLISECONDS);
   }
 
   @Override
@@ -550,20 +525,7 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void deleteTable(final TableName tableName) throws IOException {
-    Future<Void> future = deleteTableAsync(tableName);
-    try {
-      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException("Interrupted when waiting for table to be deleted");
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException)e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
-    }
+    get(deleteTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
   }
 
   @Override
@@ -641,21 +603,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public void truncateTable(final TableName tableName, final boolean preserveSplits)
       throws IOException {
-    Future<Void> future = truncateTableAsync(tableName, preserveSplits);
-    try {
-      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException("Interrupted when waiting for table " + tableName
-          + " to be enabled.");
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException) e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
-    }
+    get(truncateTableAsync(tableName, preserveSplits), syncWaitTimeout, TimeUnit.MILLISECONDS);
   }
 
   @Override
@@ -717,20 +665,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public void enableTable(final TableName tableName)
   throws IOException {
-    Future<Void> future = enableTableAsync(tableName);
-    try {
-      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException)e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
-    }
+    get(enableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
   }
 
   /**
@@ -833,20 +768,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public void disableTable(final TableName tableName)
   throws IOException {
-    Future<Void> future = disableTableAsync(tableName);
-    try {
-      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(e);
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException)e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
-    }
+    get(disableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
   }
 
   @Override
@@ -1841,43 +1763,103 @@ public class HBaseAdmin implements Admin {
     return this.conf;
   }
 
+  /**
+   * Do a get with a timeout against the passed in <code>future<code>.
+   */
+  private static <T> T get(final Future<T> future, final long timeout, final TimeUnit units)
+  throws IOException {
+    try {
+      // TODO: how long should we wait? Spin forever?
+      return future.get(timeout, units);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException("Interrupt while waiting on " + future);
+    } catch (TimeoutException e) {
+      throw new TimeoutIOException(e);
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException)e.getCause();
+      } else {
+        throw new IOException(e.getCause());
+      }
+    }
+  }
+
   @Override
-  public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+  public void createNamespace(final NamespaceDescriptor descriptor)
+  throws IOException {
+    get(createNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
+  throws IOException {
+    CreateNamespaceResponse response =
+        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection()) {
       @Override
-      public Void call(int callTimeout) throws Exception {
-        master.createNamespace(null,
+      public CreateNamespaceResponse call(int callTimeout) throws Exception {
+        return master.createNamespace(null,
           CreateNamespaceRequest.newBuilder()
             .setNamespaceDescriptor(ProtobufUtil
               .toProtoNamespaceDescriptor(descriptor)).build()
         );
-        return null;
       }
     });
+    return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
+      @Override
+      public String getOperationType() {
+        return "CREATE_NAMESPACE";
+      }
+    };
   }
 
   @Override
-  public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+  public void modifyNamespace(final NamespaceDescriptor descriptor)
+  throws IOException {
+    get(modifyNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
+  throws IOException {
+    ModifyNamespaceResponse response =
+        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection()) {
       @Override
-      public Void call(int callTimeout) throws Exception {
-        master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
+      public ModifyNamespaceResponse call(int callTimeout) throws Exception {
+        return master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
           setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
-        return null;
       }
     });
+    return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
+      @Override
+      public String getOperationType() {
+        return "MODIFY_NAMESPACE";
+      }
+    };
   }
 
   @Override
-  public void deleteNamespace(final String name) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+  public void deleteNamespace(final String name)
+  throws IOException {
+    get(deleteNamespaceAsync(name), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> deleteNamespaceAsync(final String name)
+  throws IOException {
+    DeleteNamespaceResponse response =
+        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection()) {
       @Override
-      public Void call(int callTimeout) throws Exception {
-        master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
+      public DeleteNamespaceResponse call(int callTimeout) throws Exception {
+        return master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
           setNamespaceName(name).build());
-        return null;
       }
     });
+    return new NamespaceFuture(this, name, response.getProcId()) {
+      @Override
+      public String getOperationType() {
+        return "DELETE_NAMESPACE";
+      }
+    };
   }
 
   @Override
@@ -3184,6 +3166,11 @@ public class HBaseAdmin implements Admin {
       this.tableName = tableName;
     }
 
+    @Override
+    public String toString() {
+      return getDescription();
+    }
+
     /**
      * @return the table name
      */
@@ -3222,7 +3209,7 @@ public class HBaseAdmin implements Admin {
       @Override
       public void throwTimeoutException(long elapsedTime) throws TimeoutException {
         throw new TimeoutException("The operation: " + getOperationType() + " on table: " +
-            tableName.getNameAsString() + " not completed after " + elapsedTime + "msec");
+            tableName.getNameAsString() + " has not completed after " + elapsedTime + "ms");
       }
     }
 
@@ -3344,6 +3331,34 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  protected static abstract class NamespaceFuture extends ProcedureFuture<Void> {
+    private final String namespaceName;
+
+    public NamespaceFuture(final HBaseAdmin admin, final String namespaceName, final Long procId) {
+      super(admin, procId);
+      this.namespaceName = namespaceName;
+    }
+
+    /**
+     * @return the namespace name
+     */
+    protected String getNamespaceName() {
+      return namespaceName;
+    }
+
+    /**
+     * @return the operation type like CREATE_NAMESPACE, DELETE_NAMESPACE, etc.
+     */
+    public abstract String getOperationType();
+
+    @Override
+    public String toString() {
+      return "Operation: " + getOperationType() + ", Namespace: " + getNamespaceName();
+    }
+  }
+
   @Override
   public List<SecurityCapability> getSecurityCapabilities() throws IOException {
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java
new file mode 100644
index 0000000..3b01a9e
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java
@@ -0,0 +1,44 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.coprocessor;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Thrown if a coprocessor rules we should bypass an operation
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BypassCoprocessorException extends CoprocessorException {
+  private static final long serialVersionUID = 5943889011582357043L;
+
+  /** Default Constructor */
+  public BypassCoprocessorException() {
+    super();
+  }
+
+  /**
+   * Constructs the exception and supplies a string as the message
+   * @param s - message
+   */
+  public BypassCoprocessorException(String s) {
+    super(s);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
new file mode 100644
index 0000000..97d93cc
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
@@ -0,0 +1,50 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Simple Service.
+ */
+// This is a WIP. We have Services throughout hbase. Either have all implement what is here or
+// just remove this as an experiment that did not work out.
+// TODO: Move on to guava Service after we update our guava version; later guava has nicer
+// Service implmentation.
+// TODO: Move all Services on to this one Interface.
+@InterfaceAudience.Private
+public interface Service {
+  /**
+   * Initiates service startup (if necessary), returning once the service has finished starting.
+   * @throws IOException Throws exception if already running and if we fail to start successfully.
+   */
+  void startAndWait() throws IOException;
+
+  /**
+   * @return True if this Service is running.
+   */
+  boolean isRunning();
+
+  /**
+   * Initiates service shutdown (if necessary), returning once the service has finished stopping.
+   * @throws IOException Throws exception if not running of if we fail to stop successfully.
+   */
+  void stopAndWait() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
new file mode 100644
index 0000000..f6325ea
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
@@ -0,0 +1,39 @@
+package org.apache.hadoop.hbase;
+/**
+ * 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.
+ */
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+public class ServiceNotRunningException extends HBaseIOException {
+  public ServiceNotRunningException() {
+  }
+
+  public ServiceNotRunningException(String message) {
+    super(message);
+  }
+
+  public ServiceNotRunningException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ServiceNotRunningException(Throwable cause) {
+    super(cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
index 539aea3..ee0380a 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
@@ -41,7 +41,7 @@ public class ResourceChecker {
 
   /**
    * Constructor
-   * @param tagLine - the tagLine is added to the logs. Must be be null.
+   * @param tagLine The tagLine is added to the logs. Must not be null.
    */
   public ResourceChecker(final String tagLine) {
     this.tagLine = tagLine;


[09/50] [abbrv] hbase git commit: HBASE-15044 Region normalization should be allowed when underlying namespace has quota

Posted by jm...@apache.org.
HBASE-15044 Region normalization should be allowed when underlying namespace has quota


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

Branch: refs/heads/trunk
Commit: c1b6d47e7974a5d9d75933bab9a28572e9d95c14
Parents: 07b6236
Author: tedyu <yu...@gmail.com>
Authored: Wed Dec 30 08:04:23 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Dec 30 08:04:23 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/AssignmentManager.java  | 19 ++++++--
 .../org/apache/hadoop/hbase/master/HMaster.java | 14 +++---
 .../hadoop/hbase/master/MasterServices.java     |  6 +++
 .../normalizer/EmptyNormalizationPlan.java      |  6 +++
 .../normalizer/MergeNormalizationPlan.java      |  6 +++
 .../master/normalizer/NormalizationPlan.java    | 10 ++++
 .../master/normalizer/RegionNormalizer.java     | 16 +++++++
 .../normalizer/SimpleRegionNormalizer.java      | 12 +++++
 .../normalizer/SplitNormalizationPlan.java      |  6 +++
 .../hadoop/hbase/master/TestCatalogJanitor.java |  6 +++
 .../TestSimpleRegionNormalizerOnCluster.java    | 50 +++++++++++++++-----
 .../hbase/namespace/TestNamespaceAuditor.java   | 10 ++--
 12 files changed, 135 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/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 f7f98fe..4feb2e7 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
@@ -72,8 +72,10 @@ import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.quotas.QuotaExceededException;
 import org.apache.hadoop.hbase.quotas.RegionStateListener;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
@@ -1654,9 +1656,13 @@ public class AssignmentManager {
         }
       }
       RegionLocations rl =  MetaTableAccessor.getRegionLocations(result);
-      if (rl == null) continue;
+      if (rl == null) {
+        continue;
+      }
       HRegionLocation[] locations = rl.getRegionLocations();
-      if (locations == null) continue;
+      if (locations == null) {
+        continue;
+      }
       for (HRegionLocation hrl : locations) {
         if (hrl == null) continue;
         HRegionInfo regionInfo = hrl.getRegionInfo();
@@ -1996,7 +2002,9 @@ public class AssignmentManager {
    */
   public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
       throws InterruptedException {
-    if (!regionStates.isRegionInTransition(hri)) return true;
+    if (!regionStates.isRegionInTransition(hri)) {
+      return true;
+    }
     long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTime()
         + timeOut;
     // There is already a timeout monitor on regions in transition so I
@@ -2836,7 +2844,10 @@ public class AssignmentManager {
           regionStateListener.onRegionSplit(hri);
           errorMsg = onRegionReadyToSplit(current, hri, serverName, transition);
         } catch (IOException exp) {
-          errorMsg = StringUtils.stringifyException(exp);
+            if (exp instanceof QuotaExceededException) {
+              server.getRegionNormalizer().planSkipped(hri, PlanType.SPLIT);
+            }
+            errorMsg = StringUtils.stringifyException(exp);
         }
         break;
       case SPLIT_PONR:

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/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 bdb19f4..8e51f25 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
@@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
@@ -1326,6 +1327,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return true;
   }
 
+  @VisibleForTesting
+  public RegionNormalizer getRegionNormalizer() {
+    return this.normalizer;
+  }
+
   /**
    * Perform normalization of cluster (invoked by {@link RegionNormalizerChore}).
    *
@@ -1353,11 +1359,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       Collections.shuffle(allEnabledTables);
 
       for (TableName table : allEnabledTables) {
-        if (quotaManager.getNamespaceQuotaManager() != null &&
-            quotaManager.getNamespaceQuotaManager().getState(table.getNamespaceAsString()) != null){
-          LOG.debug("Skipping normalizing " + table + " since its namespace has quota");
-          continue;
-        }
         TableDescriptor tblDesc = getTableDescriptors().getDescriptor(table);
         if (table.isSystemTable() || (tblDesc != null &&
             tblDesc.getHTableDescriptor() != null &&
@@ -1366,7 +1367,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
             + " table or doesn't have auto normalization turned on");
           continue;
         }
-        this.normalizer.computePlanForTable(table).execute(clusterConnection.getAdmin());
+        NormalizationPlan plan = this.normalizer.computePlanForTable(table);
+        plan.execute(clusterConnection.getAdmin());
       }
     }
     // If Region did not generate any plans, it means the cluster is already balanced.

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 935644f..af0e490 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
@@ -88,6 +89,11 @@ public interface MasterServices extends Server {
    * @return Master's instance of {@link MasterQuotaManager}
    */
   MasterQuotaManager getMasterQuotaManager();
+  
+  /**
+   * @return Master's instance of {@link RegionNormalizer}
+   */
+  RegionNormalizer getRegionNormalizer();
 
   /**
    * @return Master's instance of {@link ProcedureExecutor}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
index a36dd07..5aecc48 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.normalizer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 /**
  * Plan which signifies that no normalization is required,
@@ -45,4 +46,9 @@ public final class EmptyNormalizationPlan implements NormalizationPlan {
   @Override
   public void execute(Admin admin) {
   }
+
+  @Override
+  public PlanType getType() {
+    return PlanType.NONE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
index 08a58a5..e2035bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 import java.io.IOException;
 
@@ -41,6 +42,11 @@ public class MergeNormalizationPlan implements NormalizationPlan {
     this.secondRegion = secondRegion;
   }
 
+  @Override
+  public PlanType getType() {
+    return PlanType.MERGE;
+  }
+
   HRegionInfo getFirstRegion() {
     return firstRegion;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
index 96eed8c..9f866d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
@@ -26,10 +26,20 @@ import org.apache.hadoop.hbase.client.Admin;
  */
 @InterfaceAudience.Private
 public interface NormalizationPlan {
+  enum PlanType {
+    SPLIT,
+    MERGE,
+    NONE
+  }
 
   /**
    * Executes normalization plan on cluster (does actual splitting/merging work).
    * @param admin instance of Admin
    */
   void execute(Admin admin);
+
+  /**
+   * @return the type of this plan
+   */
+  PlanType getType();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
index 19abcf2..d60474d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
@@ -19,9 +19,11 @@
 package org.apache.hadoop.hbase.master.normalizer;
 
 import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 /**
  * Performs "normalization" of regions on the cluster, making sure that suboptimal
@@ -48,4 +50,18 @@ public interface RegionNormalizer {
    * @return Next (perhaps most urgent) normalization action to perform
    */
   NormalizationPlan computePlanForTable(TableName table) throws HBaseIOException;
+
+  /**
+   * Notification for the case where plan couldn't be executed due to constraint violation, such as
+   * namespace quota
+   * @param hri the region which is involved in the plan
+   * @param type type of plan
+   */
+  void planSkipped(HRegionInfo hri, PlanType type);
+  
+  /**
+   * @param type type of plan for which skipped count is to be returned
+   * @return the count of plans of specified type which were skipped
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index fe6034b..fe10bd1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.util.Triple;
 
 import java.util.ArrayList;
@@ -59,6 +60,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
   private static final Log LOG = LogFactory.getLog(SimpleRegionNormalizer.class);
   private static final int MIN_REGION_COUNT = 3;
   private MasterServices masterServices;
+  private static long[] skippedCount = new long[NormalizationPlan.PlanType.values().length];
 
   /**
    * Set the master service.
@@ -85,6 +87,16 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
     }
   };
 
+  @Override
+  public void planSkipped(HRegionInfo hri, PlanType type) {
+    skippedCount[type.ordinal()]++;
+  }
+
+  @Override
+  public long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
   /**
    * Computes next most "urgent" normalization action on the table.
    * Action may be either a split, or a merge, or no action.

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
index c96988a..b95bfb7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -42,6 +43,11 @@ public class SplitNormalizationPlan implements NormalizationPlan {
     this.splitPoint = splitPoint;
   }
 
+  @Override
+  public PlanType getType() {
+    return PlanType.SPLIT;
+  }
+
   public HRegionInfo getRegionInfo() {
     return regionInfo;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index b5f20f5..8e35bbf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLog
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
+import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -252,6 +253,11 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public RegionNormalizer getRegionNormalizer() {
+      return null;
+    }
+
+    @Override
     public MasterFileSystem getMasterFileSystem() {
       return this.mfs;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index d24023d..4fe42ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -25,11 +25,16 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.TableNamespaceManager;
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.namespace.TestNamespaceAuditor;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -63,9 +68,11 @@ public class TestSimpleRegionNormalizerOnCluster {
     // we will retry operations when PleaseHoldException is thrown
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
     TEST_UTIL.getConfiguration().setBoolean(HConstants.HBASE_NORMALIZER_ENABLED, true);
+    TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
 
     // Start a cluster of two regionservers.
     TEST_UTIL.startMiniCluster(1);
+    TestNamespaceAuditor.waitForQuotaEnabled(TEST_UTIL);
     admin = TEST_UTIL.getHBaseAdmin();
   }
 
@@ -74,11 +81,27 @@ public class TestSimpleRegionNormalizerOnCluster {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test(timeout = 60000)
+  @Test(timeout = 90000)
   @SuppressWarnings("deprecation")
   public void testRegionNormalizationSplitOnCluster() throws Exception {
-    final TableName TABLENAME =
-      TableName.valueOf("testRegionNormalizationSplitOnCluster");
+    testRegionNormalizationSplitOnCluster(false);
+    testRegionNormalizationSplitOnCluster(true);
+  }
+
+  void testRegionNormalizationSplitOnCluster(boolean limitedByQuota) throws Exception {
+    TableName TABLENAME;
+    if (limitedByQuota) {
+      String nsp = "np2";
+      NamespaceDescriptor nspDesc =
+          NamespaceDescriptor.create(nsp)
+          .addConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "5")
+          .addConfiguration(TableNamespaceManager.KEY_MAX_TABLES, "2").build();
+      admin.createNamespace(nspDesc);
+      TABLENAME = TableName.valueOf(nsp +
+        TableName.NAMESPACE_DELIM + "testRegionNormalizationSplitOnCluster");
+    } else {
+      TABLENAME = TableName.valueOf("testRegionNormalizationSplitOnCluster");
+    }
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     HMaster m = cluster.getMaster();
 
@@ -119,21 +142,26 @@ public class TestSimpleRegionNormalizerOnCluster {
 
     admin.flush(TABLENAME);
 
-    System.out.println(admin.getTableDescriptor(TABLENAME));
-
     assertEquals(5, MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), TABLENAME));
 
     // Now trigger a split and stop when the split is in progress
     Thread.sleep(5000); // to let region load to update
     m.normalizeRegions();
-
-    while (MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), TABLENAME) < 6) {
-      LOG.info("Waiting for normalization split to complete");
-      Thread.sleep(100);
+    if (limitedByQuota) {
+      long skippedSplitcnt = 0;
+      do {
+        skippedSplitcnt = m.getRegionNormalizer().getSkippedCount(PlanType.SPLIT);
+        Thread.sleep(100);
+      } while (skippedSplitcnt == 0L);
+      assert(skippedSplitcnt > 0);
+    } else {
+      while (MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), TABLENAME) < 6) {
+        LOG.info("Waiting for normalization split to complete");
+        Thread.sleep(100);
+      }
+      assertEquals(6, MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), TABLENAME));
     }
 
-    assertEquals(6, MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), TABLENAME));
-
     admin.disableTable(TABLENAME);
     admin.deleteTable(TABLENAME);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1b6d47e/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 7ded3d3..e3b841b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -115,7 +115,7 @@ public class TestNamespaceAuditor {
     conf.setClass("hbase.coprocessor.regionserver.classes", CPRegionServerObserver.class,
       RegionServerObserver.class);
     UTIL.startMiniCluster(1, 1);
-    waitForQuotaEnabled();
+    waitForQuotaEnabled(UTIL);
     ADMIN = UTIL.getHBaseAdmin();
   }
 
@@ -620,11 +620,11 @@ public class TestNamespaceAuditor {
         .getTables().size(), after.getTables().size());
   }
 
-  private static void waitForQuotaEnabled() throws Exception {
-    UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
+  public static void waitForQuotaEnabled(final HBaseTestingUtility util) throws Exception {
+    util.waitFor(60000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        HMaster master = UTIL.getHBaseCluster().getMaster();
+        HMaster master = util.getHBaseCluster().getMaster();
         if (master == null) {
           return false;
         }
@@ -638,7 +638,7 @@ public class TestNamespaceAuditor {
     UTIL.getHBaseCluster().getMaster(0).stop("Stopping to start again");
     UTIL.getHBaseCluster().waitOnMaster(0);
     UTIL.getHBaseCluster().startMaster();
-    waitForQuotaEnabled();
+    waitForQuotaEnabled(UTIL);
   }
 
   private NamespaceAuditor getQuotaManager() {


[50/50] [abbrv] hbase git commit: HBASE-15104 Occasional failures due to NotServingRegionException in IT tests (Huaxiang Sun)

Posted by jm...@apache.org.
HBASE-15104 Occasional failures due to NotServingRegionException in IT tests (Huaxiang Sun)


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

Branch: refs/heads/trunk
Commit: dc57996ca6d52b48c7f274956c0deca513ea7a83
Parents: c8b9754
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Thu Jan 14 07:59:33 2016 -0800
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Thu Jan 14 07:59:33 2016 -0800

----------------------------------------------------------------------
 .../chaos/actions/ChangeCompressionAction.java  | 21 +++++++++++++++++++-
 1 file changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dc57996c/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java
index 0d7f7ae..9c7bf45 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.io.compress.Compressor;
 
 /**
  * Action that changes the compression algorithm on a column family from a list of tables.
@@ -62,7 +63,25 @@ public class ChangeCompressionAction extends Action {
 
     // Since not every compression algorithm is supported,
     // let's use the same algorithm for all column families.
-    Algorithm algo = possibleAlgos[random.nextInt(possibleAlgos.length)];
+
+    // If an unsupported compression algorithm is chosen, pick a different one.
+    // This is to work around the issue that modifyTable() does not throw remote
+    // exception.
+    Algorithm algo;
+    do {
+      algo = possibleAlgos[random.nextInt(possibleAlgos.length)];
+
+      try {
+        Compressor c = algo.getCompressor();
+
+        // call returnCompressor() to release the Compressor
+        algo.returnCompressor(c);
+        break;
+      } catch (Throwable t) {
+        LOG.info("Performing action: Changing compression algorithms to " + algo +
+                " is not supported, pick another one");
+      }
+    } while (true);
 
     LOG.debug("Performing action: Changing compression algorithms on "
       + tableName.getNameAsString() + " to " + algo);


[17/50] [abbrv] hbase git commit: HBASE-14524 Short-circuit comparison of rows in CellComparator. (Lars Francke)

Posted by jm...@apache.org.
HBASE-14524 Short-circuit comparison of rows in CellComparator. (Lars Francke)


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

Branch: refs/heads/trunk
Commit: 9997e4ee52136a681faa825dea46bd1162d71eca
Parents: 998b937
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Jan 5 08:47:23 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Tue Jan 5 08:47:23 2016 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/CellComparator.java    | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9997e4ee/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
index 7002762..b179963 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
@@ -315,6 +315,10 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   public int compareRows(final Cell left, final Cell right) {
+    // left and right can be exactly the same at the beginning of a row
+    if (left == right) {
+      return 0;
+    }
     if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
           ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(),


[31/50] [abbrv] hbase git commit: HBASE-15073 Finer grained control over normalization actions for RegionNormalizer

Posted by jm...@apache.org.
HBASE-15073 Finer grained control over normalization actions for RegionNormalizer


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

Branch: refs/heads/trunk
Commit: d65978fceb85dd59e7fd66e2a93832a452e4c648
Parents: a9b671b
Author: tedyu <yu...@gmail.com>
Authored: Wed Jan 6 17:25:41 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jan 6 17:25:41 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   | 60 +++++++++++++-------
 .../hbase/normalizer/NormalizationPlan.java     | 45 +++++++++++++++
 .../hadoop/hbase/master/AssignmentManager.java  |  2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java | 22 ++++---
 .../normalizer/EmptyNormalizationPlan.java      |  2 +-
 .../normalizer/MergeNormalizationPlan.java      |  2 +-
 .../master/normalizer/NormalizationPlan.java    | 45 ---------------
 .../master/normalizer/RegionNormalizer.java     |  9 ++-
 .../normalizer/SimpleRegionNormalizer.java      | 14 +++--
 .../normalizer/SplitNormalizationPlan.java      |  2 +-
 .../normalizer/TestSimpleRegionNormalizer.java  | 47 +++++++++++----
 .../TestSimpleRegionNormalizerOnCluster.java    |  6 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |  5 +-
 .../src/main/ruby/shell/commands/normalize.rb   |  2 +-
 .../ruby/shell/commands/normalizer_switch.rb    |  3 +-
 15 files changed, 164 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 1bd4e07..0fb0455 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
@@ -185,13 +187,14 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
 
   /**
    * <em>INTERNAL</em> Used by shell/rest interface to access this metadata
-   * attribute which denotes if the table should be treated by region normalizer.
+   * attribute which denotes the allowed types of action (split/merge) when the table is treated
+   * by region normalizer.
    *
-   * @see #isNormalizationEnabled()
+   * @see #getDesiredNormalizationTypes()
    */
-  public static final String NORMALIZATION_ENABLED = "NORMALIZATION_ENABLED";
-  private static final Bytes NORMALIZATION_ENABLED_KEY =
-    new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
+  public static final String NORMALIZATION_MODE = "NORMALIZATION_MODE";
+  private static final Bytes NORMALIZATION_MODE_KEY =
+    new Bytes(Bytes.toBytes(NORMALIZATION_MODE));
 
   /** Default durability for HTD is USE_DEFAULT, which defaults to HBase-global default value */
   private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
@@ -220,11 +223,6 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
   public static final boolean DEFAULT_COMPACTION_ENABLED = true;
 
   /**
-   * Constant that denotes whether the table is normalized by default.
-   */
-  public static final boolean DEFAULT_NORMALIZATION_ENABLED = false;
-
-  /**
    * Constant that denotes the maximum default size of the memstore after which
    * the contents are flushed to the store files
    */
@@ -249,7 +247,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
         String.valueOf(DEFAULT_DEFERRED_LOG_FLUSH));
     DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); //use the enum name
     DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION));
-    DEFAULT_VALUES.put(NORMALIZATION_ENABLED, String.valueOf(DEFAULT_NORMALIZATION_ENABLED));
+    DEFAULT_VALUES.put(NORMALIZATION_MODE, "");
     for (String s : DEFAULT_VALUES.keySet()) {
       RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(s)));
     }
@@ -640,22 +638,42 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
   }
 
   /**
-   * Check if normalization enable flag of the table is true. If flag is
-   * false then no region normalizer won't attempt to normalize this table.
+   * Check if normalization flag of the table. If flag is
+   * empty then region normalizer won't attempt to normalize this table.
    *
-   * @return true if region normalization is enabled for this table
+   * @return List of PlanType if region normalization is enabled for this table
+   *         null means region normalization is disabled
    */
-  public boolean isNormalizationEnabled() {
-    return isSomething(NORMALIZATION_ENABLED_KEY, DEFAULT_NORMALIZATION_ENABLED);
+  public List<PlanType> getDesiredNormalizationTypes() {
+    byte [] value = getValue(NORMALIZATION_MODE_KEY);
+    if (value == null) {
+      return null;
+    }
+    String strValue = Bytes.toString(value);
+    if (strValue.isEmpty()) {
+      return null;
+    }
+    List<NormalizationPlan.PlanType> types = new ArrayList<>();
+    if (strValue.toUpperCase().contains("M")) {
+      types.add(PlanType.MERGE);
+    }
+    if (strValue.toUpperCase().contains("S")) {
+      types.add(PlanType.SPLIT);
+    }
+    return types;
   }
 
   /**
-   * Setting the table normalization enable flag.
+   * Setting the types of action for table normalization mode flag.
    *
-   * @param isEnable True if enable normalization.
-   */
-  public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
-    setValue(NORMALIZATION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+   * @param types String containing desired types of action:
+   *        "M" for region merge
+   *        "S" for region split
+   *        "MS" for region merge / split
+   */
+  public HTableDescriptor setNormalizationMode(final String types) {
+    setValue(NORMALIZATION_MODE_KEY, types == null || types.isEmpty() ? null :
+      new Bytes(Bytes.toBytes(types.toUpperCase())));
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java
new file mode 100644
index 0000000..66481e6
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/normalizer/NormalizationPlan.java
@@ -0,0 +1,45 @@
+/**
+ *
+ * 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.normalizer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+
+/**
+ * Interface for normalization plan.
+ */
+@InterfaceAudience.Private
+public interface NormalizationPlan {
+  enum PlanType {
+    SPLIT,
+    MERGE,
+    NONE
+  }
+
+  /**
+   * Executes normalization plan on cluster (does actual splitting/merging work).
+   * @param admin instance of Admin
+   */
+  void execute(Admin admin);
+
+  /**
+   * @return the type of this plan
+   */
+  PlanType getType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/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 4feb2e7..c319bb1 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
@@ -72,7 +72,7 @@ import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.QuotaExceededException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/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 2e42acb..4a9b792 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
@@ -92,7 +92,6 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
@@ -114,6 +113,8 @@ import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@@ -1323,14 +1324,21 @@ public class HMaster extends HRegionServer implements MasterServices {
 
       for (TableName table : allEnabledTables) {
         TableDescriptor tblDesc = getTableDescriptors().getDescriptor(table);
-        if (table.isSystemTable() || (tblDesc != null &&
-            tblDesc.getHTableDescriptor() != null &&
-            !tblDesc.getHTableDescriptor().isNormalizationEnabled())) {
-          LOG.debug("Skipping normalization for table: " + table + ", as it's either system"
-            + " table or doesn't have auto normalization turned on");
+        if (table.isSystemTable()) {
+          LOG.debug("Skipping normalization for table: " + table + ", as it's system table");
           continue;
         }
-        NormalizationPlan plan = this.normalizer.computePlanForTable(table);
+        List<PlanType> types = null;
+        if (tblDesc != null &&
+            tblDesc.getHTableDescriptor() != null) {
+          types = tblDesc.getHTableDescriptor().getDesiredNormalizationTypes();
+          if (types == null) {
+            LOG.debug("Skipping normalization for table: " + table + ", as it"
+                + " doesn't have auto normalization turned on");
+            continue;
+          }
+        }
+        NormalizationPlan plan = this.normalizer.computePlanForTable(table, types);
         plan.execute(clusterConnection.getAdmin());
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
index 5aecc48..29cc0c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/EmptyNormalizationPlan.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.master.normalizer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
 
 /**
  * Plan which signifies that no normalization is required,

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
index e2035bb..f3ce1d5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/MergeNormalizationPlan.java
@@ -23,7 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
deleted file mode 100644
index 9f866d3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationPlan.java
+++ /dev/null
@@ -1,45 +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.master.normalizer;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
-
-/**
- * Interface for normalization plan.
- */
-@InterfaceAudience.Private
-public interface NormalizationPlan {
-  enum PlanType {
-    SPLIT,
-    MERGE,
-    NONE
-  }
-
-  /**
-   * Executes normalization plan on cluster (does actual splitting/merging work).
-   * @param admin instance of Admin
-   */
-  void execute(Admin admin);
-
-  /**
-   * @return the type of this plan
-   */
-  PlanType getType();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
index d60474d..616098e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
@@ -18,12 +18,15 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 
 /**
  * Performs "normalization" of regions on the cluster, making sure that suboptimal
@@ -47,9 +50,11 @@ public interface RegionNormalizer {
   /**
    * Computes next optimal normalization plan.
    * @param table table to normalize
+   * @param types desired types of NormalizationPlan
    * @return Next (perhaps most urgent) normalization action to perform
    */
-  NormalizationPlan computePlanForTable(TableName table) throws HBaseIOException;
+  NormalizationPlan computePlanForTable(TableName table, List<PlanType> types)
+      throws HBaseIOException;
 
   /**
    * Notification for the case where plan couldn't be executed due to constraint violation, such as

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index fe10bd1..a035647 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -27,7 +27,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.util.Triple;
 
 import java.util.ArrayList;
@@ -60,7 +61,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
   private static final Log LOG = LogFactory.getLog(SimpleRegionNormalizer.class);
   private static final int MIN_REGION_COUNT = 3;
   private MasterServices masterServices;
-  private static long[] skippedCount = new long[NormalizationPlan.PlanType.values().length];
+  private static long[] skippedCount = new long[PlanType.values().length];
 
   /**
    * Set the master service.
@@ -102,10 +103,12 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
    * Action may be either a split, or a merge, or no action.
    *
    * @param table table to normalize
+   * @param types desired types of NormalizationPlan
    * @return normalization plan to execute
    */
   @Override
-  public NormalizationPlan computePlanForTable(TableName table) throws HBaseIOException {
+  public NormalizationPlan computePlanForTable(TableName table, List<PlanType> types)
+      throws HBaseIOException {
     if (table == null || table.isSystemTable()) {
       LOG.debug("Normalization of system table " + table + " isn't allowed");
       return EmptyNormalizationPlan.getInstance();
@@ -146,7 +149,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
 
     // now; if the largest region is >2 times large than average, we split it, split
     // is more high priority normalization action than merge.
-    if (largestRegion.getSecond() > 2 * avgRegionSize) {
+    if (types.contains(PlanType.SPLIT) && largestRegion.getSecond() > 2 * avgRegionSize) {
       LOG.debug("Table " + table + ", largest region "
         + largestRegion.getFirst().getRegionNameAsString() + " has size "
         + largestRegion.getSecond() + ", more than 2 times than avg size, splitting");
@@ -167,7 +170,8 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
     }
     Triple<HRegionInfo, Long, Integer> candidateRegion = regionsWithSize.get(candidateIdx);
     Triple<HRegionInfo, Long, Integer> candidateRegion2 = regionsWithSize.get(candidateIdx+1);
-    if (candidateRegion.getSecond() + candidateRegion2.getSecond() < avgRegionSize) {
+    if (types.contains(PlanType.MERGE) &&
+        candidateRegion.getSecond() + candidateRegion2.getSecond() < avgRegionSize) {
       LOG.debug("Table " + table + ", smallest region size: " + candidateRegion.getSecond()
         + " and its smallest neighbor size: " + candidateRegion2.getSecond()
         + ", less than the avg size, merging them");

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
index b95bfb7..76b7cc2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
@@ -23,7 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
 
 import java.io.IOException;
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
index 970af43..4395aa3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -53,6 +55,18 @@ public class TestSimpleRegionNormalizer {
   private static final Log LOG = LogFactory.getLog(TestSimpleRegionNormalizer.class);
 
   private static RegionNormalizer normalizer;
+  private static List<PlanType> bothTypes;
+  static {
+    bothTypes = new ArrayList<>();
+    bothTypes.add(PlanType.SPLIT);
+    bothTypes.add(PlanType.MERGE);
+  }
+
+  private static List<PlanType> splitType;
+  static {
+    splitType = new ArrayList<>();
+    splitType.add(PlanType.SPLIT);
+  }
 
   // mocks
   private static MasterServices masterServices;
@@ -69,7 +83,7 @@ public class TestSimpleRegionNormalizer {
     Map<byte[], Integer> regionSizes = new HashMap<>();
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
     assertTrue(plan instanceof EmptyNormalizationPlan);
   }
 
@@ -88,7 +102,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri2.getRegionName(), 15);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
     assertTrue((plan instanceof EmptyNormalizationPlan));
   }
 
@@ -114,14 +128,18 @@ public class TestSimpleRegionNormalizer {
     hris.add(hri4);
     regionSizes.put(hri4.getRegionName(), 10);
 
-
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
     assertTrue(plan instanceof EmptyNormalizationPlan);
   }
 
   @Test
   public void testMergeOfSmallRegions() throws HBaseIOException {
+    testMergeOfSmallRegions(true);
+    testMergeOfSmallRegions(false);
+  }
+
+  public void testMergeOfSmallRegions(boolean mergeDesired) throws HBaseIOException {
     TableName testTable = TableName.valueOf("testMergeOfSmallRegions");
     List<HRegionInfo> hris = new ArrayList<>();
     Map<byte[], Integer> regionSizes = new HashMap<>();
@@ -147,11 +165,16 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri5.getRegionName(), 16);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
-
-    assertTrue(plan instanceof MergeNormalizationPlan);
-    assertEquals(hri2, ((MergeNormalizationPlan) plan).getFirstRegion());
-    assertEquals(hri3, ((MergeNormalizationPlan) plan).getSecondRegion());
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable,
+      mergeDesired ? bothTypes : splitType);
+
+    if (mergeDesired) {
+      assertTrue(plan instanceof MergeNormalizationPlan);
+      assertEquals(hri2, ((MergeNormalizationPlan) plan).getFirstRegion());
+      assertEquals(hri3, ((MergeNormalizationPlan) plan).getSecondRegion());
+    } else {
+      assertTrue(plan instanceof EmptyNormalizationPlan);
+    }
   }
 
   // Test for situation illustrated in HBASE-14867
@@ -186,7 +209,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri6.getRegionName(), 2700);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
 
     assertTrue(plan instanceof MergeNormalizationPlan);
     assertEquals(hri5, ((MergeNormalizationPlan) plan).getFirstRegion());
@@ -220,7 +243,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri5.getRegionName(), 5);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
 
     assertTrue(plan instanceof EmptyNormalizationPlan);
   }
@@ -248,7 +271,7 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri4.getRegionName(), 30);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable);
+    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
 
     assertTrue(plan instanceof SplitNormalizationPlan);
     assertEquals(hri4, ((SplitNormalizationPlan) plan).getRegionInfo());

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 4fe42ed..99fb268 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.namespace.TestNamespaceAuditor;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -137,7 +137,7 @@ public class TestSimpleRegionNormalizerOnCluster {
     }
 
     HTableDescriptor htd = admin.getTableDescriptor(TABLENAME);
-    htd.setNormalizationEnabled(true);
+    htd.setNormalizationMode("MS");
     admin.modifyTable(TABLENAME, htd);
 
     admin.flush(TABLENAME);
@@ -207,7 +207,7 @@ public class TestSimpleRegionNormalizerOnCluster {
     }
 
     HTableDescriptor htd = admin.getTableDescriptor(TABLENAME);
-    htd.setNormalizationEnabled(true);
+    htd.setNormalizationMode("MS");
     admin.modifyTable(TABLENAME, htd);
 
     admin.flush(TABLENAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index c61b598..661783f 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -280,7 +280,10 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Parse arguments and update HTableDescriptor accordingly
     def parse_htd_args(htd, arg)
-      htd.setNormalizationEnabled(JBoolean.valueOf(arg.delete(NORMALIZATION_ENABLED))) if arg[NORMALIZATION_ENABLED]
+      if arg.has_key?(NORMALIZATION_MODE)
+        mode = arg.delete(NORMALIZATION_MODE)
+        htd.setValue(NORMALIZATION_MODE, mode)
+      end
     end
 
     #----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-shell/src/main/ruby/shell/commands/normalize.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/normalize.rb b/hbase-shell/src/main/ruby/shell/commands/normalize.rb
index 7e6302c..e2b3d42 100644
--- a/hbase-shell/src/main/ruby/shell/commands/normalize.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/normalize.rb
@@ -22,7 +22,7 @@ module Shell
     class Normalize < Command
       def help
         return <<-EOF
-Trigger region normalizer for all tables which have NORMALIZATION_ENABLED flag set. Returns true
+Trigger region normalizer for all tables which have NORMALIZATION_MODE flag set. Returns true
  if normalizer ran successfully, false otherwise. Note that this command has no effect
  if region normalizer is disabled (make sure it's turned on using 'normalizer_switch' command).
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d65978fc/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb b/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
index 6d959c4..ee9e2d1 100644
--- a/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/normalizer_switch.rb
@@ -23,7 +23,8 @@ module Shell
       def help
         return <<-EOF
 Enable/Disable region normalizer. Returns previous normalizer state.
-When normalizer is enabled, it handles all tables with 'NORMALIZATION_ENABLED' => true.
+When normalizer is enabled, it handles all tables with 'NORMALIZATION_MODE' flag containing
+types of normalization actions.
 Examples:
 
   hbase> normalizer_switch true


[29/50] [abbrv] hbase git commit: HBASE-12593 Tags to work with ByteBuffer.

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index faf6d81..496c7e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -338,8 +339,7 @@ public class HMobStore extends HStore {
       String fileName = MobUtils.getMobFileName(reference);
       Tag tableNameTag = MobUtils.getTableNameTag(reference);
       if (tableNameTag != null) {
-        byte[] tableName = tableNameTag.getValue();
-        String tableNameString = Bytes.toString(tableName);
+        String tableNameString = TagUtil.getValueAsString(tableNameTag);
         List<Path> locations = map.get(tableNameString);
         if (locations == null) {
           IdLock.Entry lockEntry = keyLock.getLockEntry(tableNameString.hashCode());
@@ -347,7 +347,7 @@ public class HMobStore extends HStore {
             locations = map.get(tableNameString);
             if (locations == null) {
               locations = new ArrayList<Path>(2);
-              TableName tn = TableName.valueOf(tableName);
+              TableName tn = TableName.valueOf(tableNameString);
               locations.add(MobUtils.getMobFamilyPath(conf, tn, family.getNameAsString()));
               locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils
                   .getMobRegionInfo(tn).getEncodedName(), family.getNameAsString()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/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 ccf2eb0..e553fcc 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
@@ -69,6 +69,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellScanner;
@@ -94,6 +95,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -3667,8 +3669,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       for (int i = 0; i < listSize; i++) {
         Cell cell = cells.get(i);
         List<Tag> newTags = new ArrayList<Tag>();
-        Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell.getTagsArray(),
-          cell.getTagsOffset(), cell.getTagsLength());
+        Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell);
 
         // Carry forward existing tags
 
@@ -3685,11 +3686,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // above may change when there are more tag based features in core.
         if (m.getTTL() != Long.MAX_VALUE) {
           // Add a cell TTL tag
-          newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(m.getTTL())));
+          newTags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(m.getTTL())));
         }
 
         // Rewrite the cell with the updated set of tags
-        cells.set(i, new TagRewriteCell(cell, Tag.fromList(newTags)));
+        cells.set(i, new TagRewriteCell(cell, TagUtil.fromList(newTags)));
       }
     }
   }
@@ -7073,8 +7074,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
     if (cell.getTagsLength() <= 0) return tags;
     List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags; 
-    Iterator<Tag> i =
-        CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+    Iterator<Tag> i = CellUtil.tagsIterator(cell);
     while (i.hasNext()) newTags.add(i.next());
     return newTags;
   }
@@ -7178,11 +7178,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
                 if (mutate.getTTL() != Long.MAX_VALUE) {
                   // Add the new TTL tag
-                  newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
+                  newTags.add(
+                      new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
                 }
 
                 // Rebuild tags
-                byte[] tagBytes = Tag.fromList(newTags);
+                byte[] tagBytes = TagUtil.fromList(newTags);
 
                 // allocate an empty cell once
                 newCell = new KeyValue(row.length, cell.getFamilyLength(),
@@ -7216,9 +7217,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
                 if (mutate.getTTL() != Long.MAX_VALUE) {
                   List<Tag> newTags = new ArrayList<Tag>(1);
-                  newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
+                  newTags.add(
+                      new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
                   // Add the new TTL tag
-                  newCell = new TagRewriteCell(cell, Tag.fromList(newTags));
+                  newCell = new TagRewriteCell(cell, TagUtil.fromList(newTags));
                 } else {
                   newCell = cell;
                 }
@@ -7439,7 +7441,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
               // Add the TTL tag if the mutation carried one
               if (mutation.getTTL() != Long.MAX_VALUE) {
-                newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutation.getTTL())));
+                newTags.add(
+                    new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutation.getTTL())));
               }
 
               Cell newKV = new KeyValue(row, 0, row.length,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index badbd65..8d66696 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
@@ -1779,28 +1780,24 @@ public class HStore implements Store {
    * @return true if the cell is expired
    */
   static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long now) {
-    // Do not create an Iterator or Tag objects unless the cell actually has tags.
-    if (cell.getTagsLength() > 0) {
-      // Look for a TTL tag first. Use it instead of the family setting if
-      // found. If a cell has multiple TTLs, resolve the conflict by using the
-      // first tag encountered.
-      Iterator<Tag> i = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-        cell.getTagsLength());
-      while (i.hasNext()) {
-        Tag t = i.next();
-        if (TagType.TTL_TAG_TYPE == t.getType()) {
-          // Unlike in schema cell TTLs are stored in milliseconds, no need
-          // to convert
-          long ts = cell.getTimestamp();
-          assert t.getTagLength() == Bytes.SIZEOF_LONG;
-          long ttl = Bytes.toLong(t.getBuffer(), t.getTagOffset(), t.getTagLength());
-          if (ts + ttl < now) {
-            return true;
-          }
-          // Per cell TTLs cannot extend lifetime beyond family settings, so
-          // fall through to check that
-          break;
+    // Look for a TTL tag first. Use it instead of the family setting if
+    // found. If a cell has multiple TTLs, resolve the conflict by using the
+    // first tag encountered.
+    Iterator<Tag> i = CellUtil.tagsIterator(cell);
+    while (i.hasNext()) {
+      Tag t = i.next();
+      if (TagType.TTL_TAG_TYPE == t.getType()) {
+        // Unlike in schema cell TTLs are stored in milliseconds, no need
+        // to convert
+        long ts = cell.getTimestamp();
+        assert t.getValueLength() == Bytes.SIZEOF_LONG;
+        long ttl = TagUtil.getValueAsLong(t);
+        if (ts + ttl < now) {
+          return true;
         }
+        // Per cell TTLs cannot extend lifetime beyond family settings, so
+        // fall through to check that
+        break;
       }
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 887af0a..f0723c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -658,8 +659,7 @@ public class AccessControlLists {
        return null;
      }
      List<Permission> results = Lists.newArrayList();
-     Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-        cell.getTagsLength());
+     Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
      while (tagsIterator.hasNext()) {
        Tag tag = tagsIterator.next();
        if (tag.getType() == ACL_TAG_TYPE) {
@@ -668,7 +668,12 @@ public class AccessControlLists {
          // use the builder
          AccessControlProtos.UsersAndPermissions.Builder builder = 
            AccessControlProtos.UsersAndPermissions.newBuilder();
-         ProtobufUtil.mergeFrom(builder, tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
+         if (tag.hasArray()) {
+           ProtobufUtil.mergeFrom(builder, tag.getValueArray(), tag.getValueOffset(),
+               tag.getValueLength());
+         } else {
+           ProtobufUtil.mergeFrom(builder,TagUtil.cloneValue(tag));
+         }
          ListMultimap<String,Permission> kvPerms =
            ProtobufUtil.toUsersAndPermissions(builder.build());
          // Are there permissions for this user?

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 0d8b261..bb348a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.security.access;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -34,6 +35,7 @@ import java.util.TreeSet;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
@@ -54,6 +56,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
@@ -882,15 +885,13 @@ public class AccessController extends BaseMasterAndRegionObserver
       List<Cell> newCells = Lists.newArrayList();
       for (Cell cell: e.getValue()) {
         // Prepend the supplied perms in a new ACL tag to an update list of tags for the cell
-        List<Tag> tags = Lists.newArrayList(new Tag(AccessControlLists.ACL_TAG_TYPE, perms));
-        if (cell.getTagsLength() > 0) {
-          Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell.getTagsArray(),
-            cell.getTagsOffset(), cell.getTagsLength());
-          while (tagIterator.hasNext()) {
-            tags.add(tagIterator.next());
-          }
+        List<Tag> tags = new ArrayList<Tag>();
+        tags.add(new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE, perms));
+        Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell);
+        while (tagIterator.hasNext()) {
+          tags.add(tagIterator.next());
         }
-        newCells.add(new TagRewriteCell(cell, Tag.fromList(tags)));
+        newCells.add(new TagRewriteCell(cell, TagUtil.fromList(tags)));
       }
       // This is supposed to be safe, won't CME
       e.setValue(newCells);
@@ -915,14 +916,10 @@ public class AccessController extends BaseMasterAndRegionObserver
       return;
     }
     for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
-      Cell cell = cellScanner.current();
-      if (cell.getTagsLength() > 0) {
-        Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-        while (tagsItr.hasNext()) {
-          if (tagsItr.next().getType() == AccessControlLists.ACL_TAG_TYPE) {
-            throw new AccessDeniedException("Mutation contains cell with reserved type tag");
-          }
+      Iterator<Tag> tagsItr = CellUtil.tagsIterator(cellScanner.current());
+      while (tagsItr.hasNext()) {
+        if (tagsItr.next().getType() == AccessControlLists.ACL_TAG_TYPE) {
+          throw new AccessDeniedException("Mutation contains cell with reserved type tag");
         }
       }
     }
@@ -1997,32 +1994,21 @@ public class AccessController extends BaseMasterAndRegionObserver
 
     // Collect any ACLs from the old cell
     List<Tag> tags = Lists.newArrayList();
+    List<Tag> aclTags = Lists.newArrayList();
     ListMultimap<String,Permission> perms = ArrayListMultimap.create();
     if (oldCell != null) {
-      // Save an object allocation where we can
-      if (oldCell.getTagsLength() > 0) {
-        Iterator<Tag> tagIterator = CellUtil.tagsIterator(oldCell.getTagsArray(),
-          oldCell.getTagsOffset(), oldCell.getTagsLength());
-        while (tagIterator.hasNext()) {
-          Tag tag = tagIterator.next();
-          if (tag.getType() != AccessControlLists.ACL_TAG_TYPE) {
-            // Not an ACL tag, just carry it through
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Carrying forward tag from " + oldCell + ": type " + tag.getType() +
-                " length " + tag.getTagLength());
-            }
-            tags.add(tag);
-          } else {
-            // Merge the perms from the older ACL into the current permission set
-            // TODO: The efficiency of this can be improved. Don't build just to unpack
-            // again, use the builder
-            AccessControlProtos.UsersAndPermissions.Builder builder =
-              AccessControlProtos.UsersAndPermissions.newBuilder();
-            ProtobufUtil.mergeFrom(builder, tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
-            ListMultimap<String,Permission> kvPerms =
-              ProtobufUtil.toUsersAndPermissions(builder.build());
-            perms.putAll(kvPerms);
+      Iterator<Tag> tagIterator = CellUtil.tagsIterator(oldCell);
+      while (tagIterator.hasNext()) {
+        Tag tag = tagIterator.next();
+        if (tag.getType() != AccessControlLists.ACL_TAG_TYPE) {
+          // Not an ACL tag, just carry it through
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Carrying forward tag from " + oldCell + ": type " + tag.getType()
+                + " length " + tag.getValueLength());
           }
+          tags.add(tag);
+        } else {
+          aclTags.add(tag);
         }
       }
     }
@@ -2031,7 +2017,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     byte[] aclBytes = mutation.getACL();
     if (aclBytes != null) {
       // Yes, use it
-      tags.add(new Tag(AccessControlLists.ACL_TAG_TYPE, aclBytes));
+      tags.add(new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE, aclBytes));
     } else {
       // No, use what we carried forward
       if (perms != null) {
@@ -2041,8 +2027,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         if (LOG.isTraceEnabled()) {
           LOG.trace("Carrying forward ACLs from " + oldCell + ": " + perms);
         }
-        tags.add(new Tag(AccessControlLists.ACL_TAG_TYPE,
-            ProtobufUtil.toUsersAndPermissions(perms).toByteArray()));
+        tags.addAll(aclTags);
       }
     }
 
@@ -2051,7 +2036,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       return newCell;
     }
 
-    Cell rewriteCell = new TagRewriteCell(newCell, Tag.fromList(tags));
+    Cell rewriteCell = new TagRewriteCell(newCell, TagUtil.fromList(tags));
     return rewriteCell;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
index 42d6a03..f1aec09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
@@ -42,6 +42,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -90,7 +92,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
     } catch (IOException e) {
       // We write to a byte array. No Exception can happen.
     }
-    LABELS_TABLE_TAGS[0] = new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray());
+    LABELS_TABLE_TAGS[0] = new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray());
   }
 
   public DefaultVisibilityLabelServiceImpl() {
@@ -481,42 +483,37 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
       @Override
       public boolean evaluate(Cell cell) throws IOException {
         boolean visibilityTagPresent = false;
-        // Save an object allocation where we can
-        if (cell.getTagsLength() > 0) {
-          Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-              cell.getTagsLength());
-          while (tagsItr.hasNext()) {
-            boolean includeKV = true;
-            Tag tag = tagsItr.next();
-            if (tag.getType() == VISIBILITY_TAG_TYPE) {
-              visibilityTagPresent = true;
-              int offset = tag.getTagOffset();
-              int endOffset = offset + tag.getTagLength();
-              while (offset < endOffset) {
-                Pair<Integer, Integer> result = StreamUtils
-                    .readRawVarint32(tag.getBuffer(), offset);
-                int currLabelOrdinal = result.getFirst();
-                if (currLabelOrdinal < 0) {
-                  // check for the absence of this label in the Scan Auth labels
-                  // ie. to check BitSet corresponding bit is 0
-                  int temp = -currLabelOrdinal;
-                  if (bs.get(temp)) {
-                    includeKV = false;
-                    break;
-                  }
-                } else {
-                  if (!bs.get(currLabelOrdinal)) {
-                    includeKV = false;
-                    break;
-                  }
+        Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell);
+        while (tagsItr.hasNext()) {
+          boolean includeKV = true;
+          Tag tag = tagsItr.next();
+          if (tag.getType() == VISIBILITY_TAG_TYPE) {
+            visibilityTagPresent = true;
+            int offset = tag.getValueOffset();
+            int endOffset = offset + tag.getValueLength();
+            while (offset < endOffset) {
+              Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
+              int currLabelOrdinal = result.getFirst();
+              if (currLabelOrdinal < 0) {
+                // check for the absence of this label in the Scan Auth labels
+                // ie. to check BitSet corresponding bit is 0
+                int temp = -currLabelOrdinal;
+                if (bs.get(temp)) {
+                  includeKV = false;
+                  break;
+                }
+              } else {
+                if (!bs.get(currLabelOrdinal)) {
+                  includeKV = false;
+                  break;
                 }
-                offset += result.getSecond();
-              }
-              if (includeKV) {
-                // We got one visibility expression getting evaluated to true. Good to include this
-                // KV in the result then.
-                return true;
               }
+              offset += result.getSecond();
+            }
+            if (includeKV) {
+              // We got one visibility expression getting evaluated to true. Good to include this
+              // KV in the result then.
+              return true;
             }
           }
         }
@@ -596,8 +593,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
       for (Tag tag : deleteVisTags) {
         matchFound = false;
         for (Tag givenTag : putVisTags) {
-          if (Bytes.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(),
-              givenTag.getBuffer(), givenTag.getTagOffset(), givenTag.getTagLength())) {
+          if (TagUtil.matchingValue(tag, givenTag)) {
             matchFound = true;
             break;
           }
@@ -621,10 +617,10 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
   private static void getSortedTagOrdinals(List<List<Integer>> fullTagsList, Tag tag)
       throws IOException {
     List<Integer> tagsOrdinalInSortedOrder = new ArrayList<Integer>();
-    int offset = tag.getTagOffset();
-    int endOffset = offset + tag.getTagLength();
+    int offset = tag.getValueOffset();
+    int endOffset = offset + tag.getValueLength();
     while (offset < endOffset) {
-      Pair<Integer, Integer> result = StreamUtils.readRawVarint32(tag.getBuffer(), offset);
+      Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
       tagsOrdinalInSortedOrder.add(result.getFirst());
       offset += result.getSecond();
     }
@@ -678,11 +674,11 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
           visibilityString.append(VisibilityConstants.CLOSED_PARAN).append(
               VisibilityConstants.OR_OPERATOR);
         }
-        int offset = tag.getTagOffset();
-        int endOffset = offset + tag.getTagLength();
+        int offset = tag.getValueOffset();
+        int endOffset = offset + tag.getValueLength();
         boolean expressionStart = true;
         while (offset < endOffset) {
-          Pair<Integer, Integer> result = StreamUtils.readRawVarint32(tag.getBuffer(), offset);
+          Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
           int currLabelOrdinal = result.getFirst();
           if (currLabelOrdinal < 0) {
             int temp = -currLabelOrdinal;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 5b8bdb3..b025758 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
@@ -340,8 +341,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
           Tag tag = pair.getSecond();
           if (cellVisibility == null && tag != null) {
             // May need to store only the first one
-            cellVisibility = new CellVisibility(Bytes.toString(tag.getBuffer(), tag.getTagOffset(),
-                tag.getTagLength()));
+            cellVisibility = new CellVisibility(TagUtil.getValueAsString(tag));
             modifiedTagFound = true;
           }
         }
@@ -368,14 +368,13 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
             List<Cell> updatedCells = new ArrayList<Cell>();
             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
               Cell cell = cellScanner.current();
-              List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
-                  cell.getTagsLength());
+              List<Tag> tags = CellUtil.getTags(cell);
               if (modifiedTagFound) {
                 // Rewrite the tags by removing the modified tags.
                 removeReplicationVisibilityTag(tags);
               }
               tags.addAll(visibilityTags);
-              Cell updatedCell = new TagRewriteCell(cell, Tag.fromList(tags));
+              Cell updatedCell = new TagRewriteCell(cell, TagUtil.fromList(tags));
               updatedCells.add(updatedCell);
             }
             m.getFamilyCellMap().clear();
@@ -472,28 +471,22 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       // cell visiblilty tags
       // have been modified
       Tag modifiedTag = null;
-      if (cell.getTagsLength() > 0) {
-        Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(),
-            cell.getTagsOffset(), cell.getTagsLength());
-        while (tagsIterator.hasNext()) {
-          Tag tag = tagsIterator.next();
-          if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
-            modifiedTag = tag;
-            break;
-          }
+      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
+      while (tagsIterator.hasNext()) {
+        Tag tag = tagsIterator.next();
+        if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
+          modifiedTag = tag;
+          break;
         }
       }
       pair.setFirst(true);
       pair.setSecond(modifiedTag);
       return pair;
     }
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsItr.hasNext()) {
-        if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
-          return pair;
-        }
+    Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell);
+    while (tagsItr.hasNext()) {
+      if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
+        return pair;
       }
     }
     pair.setFirst(true);
@@ -520,13 +513,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     if (isSystemOrSuperUser()) {
       return true;
     }
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsItr.hasNext()) {
-        if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
-          return false;
-        }
+    Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell);
+    while (tagsItr.hasNext()) {
+      if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
+        return false;
       }
     }
     return true;
@@ -739,21 +729,17 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     boolean authCheck = authorizationEnabled && checkAuths && !(isSystemOrSuperUser());
     tags.addAll(this.visibilityLabelService.createVisibilityExpTags(cellVisibility.getExpression(),
         true, authCheck));
-    // Save an object allocation where we can
-    if (newCell.getTagsLength() > 0) {
-      // Carry forward all other tags
-      Iterator<Tag> tagsItr = CellUtil.tagsIterator(newCell.getTagsArray(),
-          newCell.getTagsOffset(), newCell.getTagsLength());
-      while (tagsItr.hasNext()) {
-        Tag tag = tagsItr.next();
-        if (tag.getType() != TagType.VISIBILITY_TAG_TYPE
-            && tag.getType() != TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
-          tags.add(tag);
-        }
+    // Carry forward all other tags
+    Iterator<Tag> tagsItr = CellUtil.tagsIterator(newCell);
+    while (tagsItr.hasNext()) {
+      Tag tag = tagsItr.next();
+      if (tag.getType() != TagType.VISIBILITY_TAG_TYPE
+          && tag.getType() != TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
+        tags.add(tag);
       }
     }
 
-    Cell rewriteCell = new TagRewriteCell(newCell, Tag.fromList(tags));
+    Cell rewriteCell = new TagRewriteCell(newCell, TagUtil.fromList(tags));
     return rewriteCell;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
index aca4994..3db54c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
@@ -24,10 +24,12 @@ import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
@@ -79,7 +81,8 @@ public class VisibilityReplicationEndpoint implements ReplicationEndpoint {
                 byte[] modifiedVisExpression = visibilityLabelsService
                     .encodeVisibilityForReplication(visTags, serializationFormat);
                 if (modifiedVisExpression != null) {
-                  nonVisTags.add(new Tag(TagType.STRING_VIS_TAG_TYPE, modifiedVisExpression));
+                  nonVisTags
+                      .add(new ArrayBackedTag(TagType.STRING_VIS_TAG_TYPE, modifiedVisExpression));
                 }
               } catch (Exception ioe) {
                 LOG.error(
@@ -92,7 +95,7 @@ public class VisibilityReplicationEndpoint implements ReplicationEndpoint {
                 continue;
               }
               // Recreate the cell with the new tags and the existing tags
-              Cell newCell = new TagRewriteCell(cell, Tag.fromList(nonVisTags));
+              Cell newCell = new TagRewriteCell(cell, TagUtil.fromList(nonVisTags));
               newEdit.add(newCell);
             } else {
               newEdit.add(cell);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
index c725b11..1db506d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
@@ -35,11 +35,13 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -74,7 +76,7 @@ public class VisibilityUtils {
   public static final String VISIBILITY_LABEL_GENERATOR_CLASS =
       "hbase.regionserver.scan.visibility.label.generator.class";
   public static final String SYSTEM_LABEL = "system";
-  public static final Tag SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG = new Tag(
+  public static final Tag SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG = new ArrayBackedTag(
       TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE,
       VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG_VAL);
   private static final String COMMA = ",";
@@ -209,16 +211,13 @@ public class VisibilityUtils {
    */
   public static Byte extractVisibilityTags(Cell cell, List<Tag> tags) {
     Byte serializationFormat = null;
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsIterator.hasNext()) {
-        Tag tag = tagsIterator.next();
-        if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
-          serializationFormat = tag.getBuffer()[tag.getTagOffset()];
-        } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
-          tags.add(tag);
-        }
+    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
+    while (tagsIterator.hasNext()) {
+      Tag tag = tagsIterator.next();
+      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
+        serializationFormat = TagUtil.getValueAsByte(tag);
+      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
+        tags.add(tag);
       }
     }
     return serializationFormat;
@@ -239,30 +238,23 @@ public class VisibilityUtils {
   public static Byte extractAndPartitionTags(Cell cell, List<Tag> visTags,
       List<Tag> nonVisTags) {
     Byte serializationFormat = null;
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsIterator.hasNext()) {
-        Tag tag = tagsIterator.next();
-        if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
-          serializationFormat = tag.getBuffer()[tag.getTagOffset()];
-        } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
-          visTags.add(tag);
-        } else {
-          // ignore string encoded visibility expressions, will be added in replication handling
-          nonVisTags.add(tag);
-        }
+    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
+    while (tagsIterator.hasNext()) {
+      Tag tag = tagsIterator.next();
+      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
+        serializationFormat = TagUtil.getValueAsByte(tag);
+      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
+        visTags.add(tag);
+      } else {
+        // ignore string encoded visibility expressions, will be added in replication handling
+        nonVisTags.add(tag);
       }
     }
     return serializationFormat;
   }
 
   public static boolean isVisibilityTagsPresent(Cell cell) {
-    if (cell.getTagsLength() == 0) {
-      return false;
-    }
-    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-        cell.getTagsLength());
+    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
     while (tagsIterator.hasNext()) {
       Tag tag = tagsIterator.next();
       if (tag.getType() == VISIBILITY_TAG_TYPE) {
@@ -322,7 +314,7 @@ public class VisibilityUtils {
     if (node.isSingleNode()) {
       getLabelOrdinals(node, labelOrdinals, auths, checkAuths, ordinalProvider);
       writeLabelOrdinalsToStream(labelOrdinals, dos);
-      tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
+      tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
       baos.reset();
     } else {
       NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
@@ -330,14 +322,14 @@ public class VisibilityUtils {
         for (ExpressionNode child : nlNode.getChildExps()) {
           getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
           writeLabelOrdinalsToStream(labelOrdinals, dos);
-          tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
+          tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
           baos.reset();
           labelOrdinals.clear();
         }
       } else {
         getLabelOrdinals(nlNode, labelOrdinals, auths, checkAuths, ordinalProvider);
         writeLabelOrdinalsToStream(labelOrdinals, dos);
-        tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
+        tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
         baos.reset();
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
index 5df7394..b212fe6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
@@ -339,12 +340,10 @@ public class WALPrettyPrinter {
     stringMap.put("vlen", cell.getValueLength());
     if (cell.getTagsLength() > 0) {
       List<String> tagsString = new ArrayList<String>();
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
+      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
       while (tagsIterator.hasNext()) {
         Tag tag = tagsIterator.next();
-        tagsString.add((tag.getType()) + ":"
-            + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
+        tagsString.add((tag.getType()) + ":" + Bytes.toStringBinary(TagUtil.cloneValue(tag)));
       }
       stringMap.put("tag", tagsString);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 30629a3..821b995 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -1405,7 +1405,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
           byte[] tag = generateData(this.rand, TAG_LENGTH);
           Tag[] tags = new Tag[opts.noOfTags];
           for (int n = 0; n < opts.noOfTags; n++) {
-            Tag t = new Tag((byte) n, tag);
+            Tag t = new ArrayBackedTag((byte) n, tag);
             tags[n] = t;
           }
           KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,
@@ -1493,7 +1493,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
           byte[] tag = generateData(this.rand, TAG_LENGTH);
           Tag[] tags = new Tag[opts.noOfTags];
           for (int n = 0; n < opts.noOfTags; n++) {
-            Tag t = new Tag((byte) n, tag);
+            Tag t = new ArrayBackedTag((byte) n, tag);
             tags[n] = t;
           }
           KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
index f83590a..1647e97 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -103,11 +104,11 @@ public class TestResultSizeEstimation {
     Table table = TEST_UTIL.createTable(TABLE, FAMILIES);
     Put p = new Put(ROW1);
     p.add(new KeyValue(ROW1, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE,
-      new Tag[] { new Tag((byte)1, new byte[TAG_DATA_SIZE]) } ));
+      new Tag[] { new ArrayBackedTag((byte)1, new byte[TAG_DATA_SIZE]) } ));
     table.put(p);
     p = new Put(ROW2);
     p.add(new KeyValue(ROW2, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE,
-      new Tag[] { new Tag((byte)1, new byte[TAG_DATA_SIZE]) } ));
+      new Tag[] { new ArrayBackedTag((byte)1, new byte[TAG_DATA_SIZE]) } ));
     table.put(p);
 
     Scan s = new Scan();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
index ce48ca1..00969b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeSeeker;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFileBlock.Writer.BufferGrabbingByteArrayOutputStream;
@@ -136,10 +137,10 @@ public class TestDataBlockEncoders {
     } else {
       byte[] metaValue1 = Bytes.toBytes("metaValue1");
       byte[] metaValue2 = Bytes.toBytes("metaValue2");
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue1) }));
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue2) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue1) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue2) }));
     }
     testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
   }
@@ -160,10 +161,10 @@ public class TestDataBlockEncoders {
     if (includesTags) {
       byte[] metaValue1 = Bytes.toBytes("metaValue1");
       byte[] metaValue2 = Bytes.toBytes("metaValue2");
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue1) }));
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue2) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue1) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue2) }));
     } else {
       kvList.add(new KeyValue(row, family, qualifier, -1l, Type.Put, value));
       kvList.add(new KeyValue(row, family, qualifier, -2l, Type.Put, value));
@@ -416,10 +417,10 @@ public class TestDataBlockEncoders {
     byte[] value0 = new byte[] { 'd' };
     byte[] value1 = new byte[] { 0x00 };
     if (includesTags) {
-      kvList.add(new KeyValue(row, family, qualifier0, 0, value0, new Tag[] { new Tag((byte) 1,
-          "value1") }));
-      kvList.add(new KeyValue(row, family, qualifier1, 0, value1, new Tag[] { new Tag((byte) 1,
-          "value1") }));
+      kvList.add(new KeyValue(row, family, qualifier0, 0, value0,
+          new Tag[] { new ArrayBackedTag((byte) 1, "value1") }));
+      kvList.add(new KeyValue(row, family, qualifier1, 0, value1,
+          new Tag[] { new ArrayBackedTag((byte) 1, "value1") }));
     } else {
       kvList.add(new KeyValue(row, family, qualifier0, 0, Type.Put, value0));
       kvList.add(new KeyValue(row, family, qualifier1, 0, Type.Put, value1));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
index ce66e82..0869df6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -148,7 +149,7 @@ public class TestEncodedSeekers {
         byte[] value = dataGenerator.generateRandomSizeValue(key, col);
         if (includeTags) {
           Tag[] tag = new Tag[1];
-          tag[0] = new Tag((byte) 1, "Visibility");
+          tag[0] = new ArrayBackedTag((byte) 1, "Visibility");
           KeyValue kv = new KeyValue(key, CF_BYTES, col, HConstants.LATEST_TIMESTAMP, value, tag);
           put.add(kv);
         } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
index 031bf25..fd9b90b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder.EncodedSeeker;
@@ -280,7 +281,7 @@ public class TestPrefixTreeEncoding {
           kvset.add(kv);
         } else {
           KeyValue kv = new KeyValue(getRowKey(batchId, i), CF_BYTES, getQualifier(j), 0l,
-              getValue(batchId, i, j), new Tag[] { new Tag((byte) 1, "metaValue1") });
+              getValue(batchId, i, j), new Tag[] { new ArrayBackedTag((byte) 1, "metaValue1") });
           kvset.add(kv);
         }
       }
@@ -308,7 +309,7 @@ public class TestPrefixTreeEncoding {
           kvset.add(kv);
         } else {
           KeyValue kv = new KeyValue(getRowKey(batchId, i), CF_BYTES, getQualifier(j), 0l,
-              getValue(batchId, i, j), new Tag[] { new Tag((byte) 1, "metaValue1") });
+              getValue(batchId, i, j), new Tag[] { new ArrayBackedTag((byte) 1, "metaValue1") });
           kvset.add(kv);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index ccf59a4..5158e35 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -39,6 +39,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -385,7 +386,7 @@ public class TestCacheOnWrite {
       byte[] value = RandomKeyValueUtil.randomValue(rand);
       KeyValue kv;
       if(useTags) {
-        Tag t = new Tag((byte) 1, "visibility");
+        Tag t = new ArrayBackedTag((byte) 1, "visibility");
         List<Tag> tagList = new ArrayList<Tag>();
         tagList.add(t);
         Tag[] tags = new Tag[1];
@@ -434,7 +435,7 @@ public class TestCacheOnWrite {
           String valueStr = "value_" + rowStr + "_" + qualStr;
           for (int iTS = 0; iTS < 5; ++iTS) {
             if (useTags) {
-              Tag t = new Tag((byte) 1, "visibility");
+              Tag t = new ArrayBackedTag((byte) 1, "visibility");
               Tag[] tags = new Tag[1];
               tags[0] = t;
               KeyValue kv = new KeyValue(Bytes.toBytes(rowStr), cfBytes, Bytes.toBytes(qualStr),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 929ad8a..66fb49c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
@@ -169,7 +170,7 @@ public class TestHFile  {
     for (int i = start; i < (start + n); i++) {
       String key = String.format(localFormatter, Integer.valueOf(i));
       if (useTags) {
-        Tag t = new Tag((byte) 1, "myTag1");
+        Tag t = new ArrayBackedTag((byte) 1, "myTag1");
         Tag[] tags = new Tag[1];
         tags[0] = t;
         kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/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 12fb584..4ee7f5b 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@@ -167,8 +168,8 @@ public class TestHFileBlock {
       if (!useTag) {
         keyValues.add(new KeyValue(row, family, qualifier, timestamp, value));
       } else {
-        keyValues.add(new KeyValue(row, family, qualifier, timestamp, value, new Tag[] { new Tag(
-            (byte) 1, Bytes.toBytes("myTagVal")) }));
+        keyValues.add(new KeyValue(row, family, qualifier, timestamp, value,
+            new Tag[] { new ArrayBackedTag((byte) 1, Bytes.toBytes("myTagVal")) }));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index 15aa912..c7eb11b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
@@ -140,7 +141,7 @@ public class TestHFileWriterV3 {
         for (int j = 0; j < 1 + rand.nextInt(4); j++) {
           byte[] tagBytes = new byte[16];
           rand.nextBytes(tagBytes);
-          tags.add(new Tag((byte) 1, tagBytes));
+          tags.add(new ArrayBackedTag((byte) 1, tagBytes));
         }
         keyValue = new KeyValue(keyBytes, null, null, HConstants.LATEST_TIMESTAMP,
             valueBytes, tags);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
index a17368c..90e398d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -84,7 +85,7 @@ public class TestReseekTo {
             Bytes.toBytes(value));
         writer.append(kv);
       } else if (tagUsage == TagUsage.ONLY_TAG) {
-        Tag t = new Tag((byte) 1, "myTag1");
+        Tag t = new ArrayBackedTag((byte) 1, "myTag1");
         Tag[] tags = new Tag[1];
         tags[0] = t;
         kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
@@ -92,7 +93,7 @@ public class TestReseekTo {
         writer.append(kv);
       } else {
         if (key % 4 == 0) {
-          Tag t = new Tag((byte) 1, "myTag1");
+          Tag t = new ArrayBackedTag((byte) 1, "myTag1");
           Tag[] tags = new Tag[1];
           tags[0] = t;
           kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
index c1d91ec..6eead71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.OffheapKeyValue;
 import org.apache.hadoop.hbase.ShareableMemory;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -88,7 +90,7 @@ public class TestSeekTo {
       return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
           Bytes.toBytes("value"));
     } else if (tagUsage == TagUsage.ONLY_TAG) {
-      Tag t = new Tag((byte) 1, "myTag1");
+      Tag t = new ArrayBackedTag((byte) 1, "myTag1");
       Tag[] tags = new Tag[1];
       tags[0] = t;
       return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
@@ -100,7 +102,7 @@ public class TestSeekTo {
             Bytes.toBytes("qualifier"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
       } else {
         switchKVs = false;
-        Tag t = new Tag((byte) 1, "myTag1");
+        Tag t = new ArrayBackedTag((byte) 1, "myTag1");
         Tag[] tags = new Tag[1];
         tags[0] = t;
         return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
@@ -174,11 +176,10 @@ public class TestSeekTo {
     assertEquals("i", toRowStr(scanner.getCell()));
     Cell cell = scanner.getCell();
     if (tagUsage != TagUsage.NO_TAG && cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
+      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
       while (tagsIterator.hasNext()) {
         Tag next = tagsIterator.next();
-        assertEquals("myTag1", Bytes.toString(next.getValue()));
+        assertEquals("myTag1", Bytes.toString(TagUtil.cloneValue(next)));
       }
     }
     assertTrue(scanner.seekBefore(toKV("k", tagUsage)));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 47b6b5c..ef02431 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
@@ -57,7 +58,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
@@ -190,7 +190,8 @@ public class TestHMobStore {
 
     String targetPathName = MobUtils.formatDate(currentDate);
     byte[] referenceValue = Bytes.toBytes(targetPathName + Path.SEPARATOR + mobFilePath.getName());
-    Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName().getName());
+    Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE,
+        store.getTableName().getName());
     KeyValue kv1 = new KeyValue(row, family, qf1, Long.MAX_VALUE, referenceValue);
     KeyValue kv2 = new KeyValue(row, family, qf2, Long.MAX_VALUE, referenceValue);
     KeyValue kv3 = new KeyValue(row2, family, qf3, Long.MAX_VALUE, referenceValue);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 4582e31..cef92a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -96,7 +96,7 @@ import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionTooBusyException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Append;
@@ -6335,16 +6335,16 @@ public class TestHRegion {
       long now = EnvironmentEdgeManager.currentTime();
       // Add a cell that will expire in 5 seconds via cell TTL
       region.put(new Put(row).add(new KeyValue(row, fam1, q1, now,
-        HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
+        HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
           // TTL tags specify ts in milliseconds
-          new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
+          new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
       // Add a cell that will expire after 10 seconds via family setting
       region.put(new Put(row).addColumn(fam1, q2, now, HConstants.EMPTY_BYTE_ARRAY));
       // Add a cell that will expire in 15 seconds via cell TTL
       region.put(new Put(row).add(new KeyValue(row, fam1, q3, now + 10000 - 1,
-        HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
+        HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
           // TTL tags specify ts in milliseconds
-          new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
+          new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
       // Add a cell that will expire in 20 seconds via family setting
       region.put(new Put(row).addColumn(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY));
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
index 1bcb7c9..3c062f8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
@@ -31,9 +31,11 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -86,7 +88,7 @@ public class TestStoreFileScannerWithTagCompression {
           kv.getRowLength()));
       List<Tag> tags = KeyValueUtil.ensureKeyValue(kv).getTags();
       assertEquals(1, tags.size());
-      assertEquals("tag3", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag3", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
     } finally {
       s.close();
     }
@@ -97,9 +99,9 @@ public class TestStoreFileScannerWithTagCompression {
     byte[] qualifier = Bytes.toBytes("q");
     long now = System.currentTimeMillis();
     byte[] b = Bytes.toBytes("k1");
-    Tag t1 = new Tag((byte) 1, "tag1");
-    Tag t2 = new Tag((byte) 2, "tag2");
-    Tag t3 = new Tag((byte) 3, "tag3");
+    Tag t1 = new ArrayBackedTag((byte) 1, "tag1");
+    Tag t2 = new ArrayBackedTag((byte) 2, "tag2");
+    Tag t3 = new ArrayBackedTag((byte) 3, "tag3");
     try {
       writer.append(new KeyValue(b, fam, qualifier, now, b, new Tag[] { t1 }));
       b = Bytes.toBytes("k3");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
index a85e479..0f7f23a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Durability;
@@ -325,7 +327,7 @@ public class TestTags {
             if (CellUtil.matchingRow(current, row)) {
               assertEquals(1, TestCoprocessorForTags.tags.size());
               Tag tag = TestCoprocessorForTags.tags.get(0);
-              assertEquals(bigTagLen, tag.getTagLength());
+              assertEquals(bigTagLen, tag.getValueLength());
             } else {
               assertEquals(0, TestCoprocessorForTags.tags.size());
             }
@@ -350,7 +352,7 @@ public class TestTags {
             if (CellUtil.matchingRow(current, row)) {
               assertEquals(1, TestCoprocessorForTags.tags.size());
               Tag tag = TestCoprocessorForTags.tags.get(0);
-              assertEquals(bigTagLen, tag.getTagLength());
+              assertEquals(bigTagLen, tag.getValueLength());
             } else {
               assertEquals(0, TestCoprocessorForTags.tags.size());
             }
@@ -403,7 +405,7 @@ public class TestTags {
       List<Tag> tags = TestCoprocessorForTags.tags;
       assertEquals(3L, Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
       assertEquals(1, tags.size());
-      assertEquals("tag1", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
 
@@ -421,7 +423,7 @@ public class TestTags {
       // We cannot assume the ordering of tags
       List<String> tagValues = new ArrayList<String>();
       for (Tag tag: tags) {
-        tagValues.add(Bytes.toString(tag.getValue()));
+        tagValues.add(Bytes.toString(TagUtil.cloneValue(tag)));
       }
       assertTrue(tagValues.contains("tag1"));
       assertTrue(tagValues.contains("tag2"));
@@ -445,7 +447,7 @@ public class TestTags {
       tags = TestCoprocessorForTags.tags;
       assertEquals(4L, Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
       assertEquals(1, tags.size());
-      assertEquals("tag2", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag2", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
 
@@ -466,7 +468,7 @@ public class TestTags {
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;
       assertEquals(1, tags.size());
-      assertEquals("tag1", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
 
@@ -483,7 +485,7 @@ public class TestTags {
       // We cannot assume the ordering of tags
       tagValues.clear();
       for (Tag tag: tags) {
-        tagValues.add(Bytes.toString(tag.getValue()));
+        tagValues.add(Bytes.toString(TagUtil.cloneValue(tag)));
       }
       assertTrue(tagValues.contains("tag1"));
       assertTrue(tagValues.contains("tag2"));
@@ -506,7 +508,7 @@ public class TestTags {
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;
       assertEquals(1, tags.size());
-      assertEquals("tag2", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag2", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
     } finally {
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
@@ -569,7 +571,7 @@ public class TestTags {
             if (cf == null) {
               cf = CellUtil.cloneFamily(kv);
             }
-            Tag tag = new Tag((byte) 1, attribute);
+            Tag tag = new ArrayBackedTag((byte) 1, attribute);
             List<Tag> tagList = new ArrayList<Tag>();
             tagList.add(tag);
 
@@ -611,7 +613,7 @@ public class TestTags {
           CellScanner cellScanner = result.cellScanner();
           if (cellScanner.advance()) {
             Cell cell = cellScanner.current();
-            tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
+            tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(),
                 cell.getTagsLength());
           }
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
index 0450904..104f897 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
@@ -24,9 +24,10 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -108,7 +109,7 @@ public class TestKeyValueCompression {
     byte[] value = Bytes.toBytes("myValue");
     List<Tag> tags = new ArrayList<Tag>(noOfTags);
     for (int i = 1; i <= noOfTags; i++) {
-      tags.add(new Tag((byte) i, Bytes.toBytes("tagValue" + i)));
+      tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i)));
     }
     return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
index 501fdda..e834ac8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.Codec.Decoder;
 import org.apache.hadoop.hbase.codec.Codec.Encoder;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
@@ -69,7 +71,7 @@ public class TestWALCellCodecWithCompression {
     KeyValue kv = (KeyValue) decoder.current();
     List<Tag> tags = kv.getTags();
     assertEquals(1, tags.size());
-    assertEquals("tagValue1", Bytes.toString(tags.get(0).getValue()));
+    assertEquals("tagValue1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
     decoder.advance();
     kv = (KeyValue) decoder.current();
     tags = kv.getTags();
@@ -78,8 +80,8 @@ public class TestWALCellCodecWithCompression {
     kv = (KeyValue) decoder.current();
     tags = kv.getTags();
     assertEquals(2, tags.size());
-    assertEquals("tagValue1", Bytes.toString(tags.get(0).getValue()));
-    assertEquals("tagValue2", Bytes.toString(tags.get(1).getValue()));
+    assertEquals("tagValue1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
+    assertEquals("tagValue2", Bytes.toString(TagUtil.cloneValue(tags.get(1))));
   }
 
   private KeyValue createKV(int noOfTags) {
@@ -89,7 +91,7 @@ public class TestWALCellCodecWithCompression {
     byte[] value = Bytes.toBytes("myValue");
     List<Tag> tags = new ArrayList<Tag>(noOfTags);
     for (int i = 1; i <= noOfTags; i++) {
-      tags.add(new Tag((byte) i, Bytes.toBytes("tagValue" + i)));
+      tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i)));
     }
     return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
index 988373f..8bfdc2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -209,7 +211,7 @@ public class TestReplicationWithTags {
             if (cf == null) {
               cf = CellUtil.cloneFamily(kv);
             }
-            Tag tag = new Tag(TAG_TYPE, attribute);
+            Tag tag = new ArrayBackedTag(TAG_TYPE, attribute);
             List<Tag> tagList = new ArrayList<Tag>();
             tagList.add(tag);
 
@@ -238,7 +240,7 @@ public class TestReplicationWithTags {
         // Check tag presence in the 1st cell in 1st Result
         if (!results.isEmpty()) {
           Cell cell = results.get(0);
-          tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+          tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 8ecc6e3..9f20c11 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Connection;
@@ -2516,7 +2517,7 @@ public class TestAccessController extends SecureTestUtil {
             Table t = conn.getTable(TEST_TABLE);) {
           KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
             HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY,
-            new Tag[] { new Tag(AccessControlLists.ACL_TAG_TYPE,
+            new Tag[] { new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE,
               ProtobufUtil.toUsersAndPermissions(USER_OWNER.getShortName(),
                 new Permission(Permission.Action.READ)).toByteArray()) });
           t.put(new Put(TEST_ROW).add(kv));


[45/50] [abbrv] hbase git commit: HBASE-15083 Gets from Multiactions are not counted in metrics for gets

Posted by jm...@apache.org.
HBASE-15083 Gets from Multiactions are not counted in metrics for gets


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

Branch: refs/heads/trunk
Commit: 8ee9158b5475b7d5bb6ba646420403820fac1d63
Parents: da932ee
Author: chenheng <ch...@apache.org>
Authored: Mon Jan 11 10:44:50 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Tue Jan 12 14:18:36 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/RSRpcServices.java  | 18 +++++++++++++-----
 1 file changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8ee9158b/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 5d9b90f..7eaadc2 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
@@ -699,11 +699,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           continue;
         }
         if (action.hasGet()) {
-          Get get = ProtobufUtil.toGet(action.getGet());
-          if (context != null) {
-            r = get(get, ((HRegion) region), closeCallBack, context);
-          } else {
-            r = region.get(get);
+          long before = EnvironmentEdgeManager.currentTime();
+          try {
+            Get get = ProtobufUtil.toGet(action.getGet());
+            if (context != null) {
+              r = get(get, ((HRegion) region), closeCallBack, context);
+            } else {
+              r = region.get(get);
+            }
+          } finally {
+            if (regionServer.metricsRegionServer != null) {
+              regionServer.metricsRegionServer.updateGet(
+                EnvironmentEdgeManager.currentTime() - before);
+            }
           }
         } else if (action.hasServiceCall()) {
           resultOrExceptionBuilder = ResultOrException.newBuilder();


[12/50] [abbrv] hbase git commit: HBASE-14888 ClusterSchema: Add Namespace Operations

Posted by jm...@apache.org.
HBASE-14888  ClusterSchema: Add Namespace Operations


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

Branch: refs/heads/trunk
Commit: c5f3d17ae3a61cbf77cab89cddd8303e20e5e734
Parents: f01a388
Author: stack <st...@apache.org>
Authored: Sat Jan 2 16:16:15 2016 -0800
Committer: stack <st...@apache.org>
Committed: Sat Jan 2 16:16:56 2016 -0800

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  |   2 +-
 .../java/org/apache/hadoop/hbase/Service.java   |  50 +++
 .../hbase/ServiceNotRunningException.java       |  39 +++
 .../apache/hadoop/hbase/ResourceChecker.java    |   2 +-
 .../hadoop/hbase/master/ClusterSchema.java      | 153 +++++++++
 .../hbase/master/ClusterSchemaException.java    |  37 +++
 .../hbase/master/ClusterSchemaService.java      |  27 ++
 .../hbase/master/ClusterSchemaServiceImpl.java  | 164 ++++++++++
 .../org/apache/hadoop/hbase/master/HMaster.java | 314 ++++++-------------
 .../hadoop/hbase/master/MasterRpcServices.java  |  10 +-
 .../hadoop/hbase/master/MasterServices.java     |  78 +----
 .../hadoop/hbase/master/ProcedureFuture.java    | 132 ++++++++
 .../hbase/master/TableNamespaceManager.java     |  68 ++--
 .../procedure/CreateNamespaceProcedure.java     |   2 +-
 .../procedure/DeleteNamespaceProcedure.java     |   2 +-
 .../procedure/ModifyNamespaceProcedure.java     |   3 +-
 .../hbase/namespace/NamespaceStateManager.java  |   6 +-
 .../hbase/regionserver/HRegionServer.java       |   3 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |   2 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |  61 +---
 .../hbase/master/TestMasterNoCluster.java       |   8 -
 21 files changed, 762 insertions(+), 401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 0ef2a17..3498590 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -398,7 +398,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       synchronized (this) {
         if (batchPool == null) {
           this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256),
-              conf.getInt("hbase.hconnection.threads.core", 256), "-shared-", null);
+              conf.getInt("hbase.hconnection.threads.core", 256), "-shared", null);
           this.cleanupPool = true;
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
new file mode 100644
index 0000000..97d93cc
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
@@ -0,0 +1,50 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Simple Service.
+ */
+// This is a WIP. We have Services throughout hbase. Either have all implement what is here or
+// just remove this as an experiment that did not work out.
+// TODO: Move on to guava Service after we update our guava version; later guava has nicer
+// Service implmentation.
+// TODO: Move all Services on to this one Interface.
+@InterfaceAudience.Private
+public interface Service {
+  /**
+   * Initiates service startup (if necessary), returning once the service has finished starting.
+   * @throws IOException Throws exception if already running and if we fail to start successfully.
+   */
+  void startAndWait() throws IOException;
+
+  /**
+   * @return True if this Service is running.
+   */
+  boolean isRunning();
+
+  /**
+   * Initiates service shutdown (if necessary), returning once the service has finished stopping.
+   * @throws IOException Throws exception if not running of if we fail to stop successfully.
+   */
+  void stopAndWait() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
new file mode 100644
index 0000000..f6325ea
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
@@ -0,0 +1,39 @@
+package org.apache.hadoop.hbase;
+/**
+ * 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.
+ */
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+public class ServiceNotRunningException extends HBaseIOException {
+  public ServiceNotRunningException() {
+  }
+
+  public ServiceNotRunningException(String message) {
+    super(message);
+  }
+
+  public ServiceNotRunningException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ServiceNotRunningException(Throwable cause) {
+    super(cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
index 539aea3..ee0380a 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
@@ -41,7 +41,7 @@ public class ResourceChecker {
 
   /**
    * Constructor
-   * @param tagLine - the tagLine is added to the logs. Must be be null.
+   * @param tagLine The tagLine is added to the logs. Must not be null.
    */
   public ResourceChecker(final String tagLine) {
     this.tagLine = tagLine;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
new file mode 100644
index 0000000..e632230
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
@@ -0,0 +1,153 @@
+/**
+ * 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 java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.List;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * View and edit the current cluster schema. Use this API making any modification to
+ * namespaces, tables, etc.
+ *
+ * <h2>Implementation Notes</h2>
+ * Nonces are for when operation is non-idempotent to ensure once-only semantic, even
+ * across process failures.
+ */
+// ClusterSchema is introduced to encapsulate schema modification. Currently the different aspects
+// are spread about the code base. This effort is about cleanup, shutting down access, and
+// coalescing common code. In particular, we'd contain filesystem modification. Other
+// benefits are to make all schema modification work the same way (one way to do an operation only
+// rather than the current approach where how an operation is done varies with context) and to make
+// it so clusterschema modification can stand apart from Master to faciliate standalone
+// testing. It is part of the filesystem refactor project that undoes the dependency on a
+// layout in HDFS that mimics our model of tables have regions have column families have files.
+// With this Interface in place, with all modifications going via this route where no filesystem
+// particulars are exposed, redoing our internals will take less effort.
+//
+// Currently ClusterSchema Interface will include namespace and table manipulation. Ideally a
+// form of this Interface will go all the ways down to the file manipulation level but currently
+// TBD.
+//
+// ClusterSchema is private to the Master; only the Master knows current cluster state and has
+// means of editing/altering it.
+//
+// TODO: Remove Server argument when MasterServices are passed.
+// TODO: We return Future<ProcedureInfo> in the below from most methods. It may change to return
+// a ProcedureFuture subsequently.
+@InterfaceAudience.Private
+public interface ClusterSchema {
+  /**
+   * Timeout for cluster operations in milliseconds.
+   */
+  public static final String HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY =
+      "hbase.master.cluster.schema.operation.timeout";
+  /**
+   * Default operation timeout in milliseconds.
+   */
+  public static final int DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT =
+      5 * 60 * 1000;
+
+  /**
+   * Utility method that will wait {@link #HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY}
+   * timeout and if exceptions, THROWs the exception doing conversion so palatable outside Master:
+   * i.e. {@link InterruptedException} becomes {@link InterruptedIOException} and so on.
+   *
+   * <<Utility>>
+   *
+   * @param future Future to wait on.
+   * @return On completion, info on the procedure that ran.
+   * @throws IOException
+   */
+  // TODO: Where to put this utility? It goes away?
+  ProcedureInfo get(final Future<ProcedureInfo> future) throws IOException;
+
+  /**
+   * For internals use only. Do not use! Provisionally part of this Interface.
+   * Prefer the high-level APIs available elsewhere in this API.
+   * @return Instance of {@link TableNamespaceManager}
+   */
+  // TODO: Remove from here. Keep internal. This Interface is too high-level to host this accessor.
+  TableNamespaceManager getTableNamespaceManager();
+
+  /**
+   * Create a new Namespace.
+   * @param namespaceDescriptor descriptor for new Namespace
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return Operation Future.
+   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   * as well as {@link IOException}
+   */
+  Future<ProcedureInfo> createNamespace(NamespaceDescriptor namespaceDescriptor, long nonceGroup,
+      long nonce)
+  throws IOException;
+
+  /**
+   * Modify an existing Namespace.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return Operation Future.
+   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   * as well as {@link IOException}
+   */
+  Future<ProcedureInfo> modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup,
+      long nonce)
+  throws IOException;
+
+  /**
+   * Delete an existing Namespace.
+   * Only empty Namespaces (no tables) can be removed.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @return Operation Future.
+   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   * as well as {@link IOException}
+   */
+  Future<ProcedureInfo> deleteNamespace(String name, long nonceGroup, long nonce)
+  throws IOException;
+
+  /**
+   * Get a Namespace
+   * @param name Name of the Namespace
+   * @return Namespace descriptor for <code>name</code>
+   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+   * as well as {@link IOException}
+   */
+  // No Future here because presumption is that the request will go against cached metadata so
+  // return immediately -- no need of running a Procedure.
+  NamespaceDescriptor getNamespace(String name) throws IOException;
+
+  /**
+   * Get all Namespaces
+   * @return All Namespace descriptors
+   * @throws IOException
+   */
+  List<NamespaceDescriptor> getNamespaces() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
new file mode 100644
index 0000000..62892b6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
@@ -0,0 +1,37 @@
+/**
+ * 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.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+public class ClusterSchemaException extends HBaseIOException {
+  public ClusterSchemaException(String message) {
+    super(message);
+  }
+
+  public ClusterSchemaException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ClusterSchemaException(Throwable cause) {
+    super(cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
new file mode 100644
index 0000000..43353ba
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
@@ -0,0 +1,27 @@
+/**
+ * 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.hadoop.hbase.Service;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Mixes in ClusterSchema and Service
+ */
+@InterfaceAudience.Private
+public interface ClusterSchemaService extends ClusterSchema, Service {}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
new file mode 100644
index 0000000..c7f4692
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
@@ -0,0 +1,164 @@
+/**
+ * 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 java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ServiceNotRunningException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
+import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+
+@InterfaceAudience.Private
+class ClusterSchemaServiceImpl implements ClusterSchemaService {
+  private boolean running = false;
+  private final TableNamespaceManager tableNamespaceManager;
+  private final MasterServices masterServices;
+  private final int clusterSchemaOperationTimeoutInMillis;
+  private final static List<NamespaceDescriptor> EMPTY_NAMESPACE_LIST =
+    Collections.unmodifiableList(new ArrayList<NamespaceDescriptor>(0));
+
+  ClusterSchemaServiceImpl(final MasterServices masterServices) {
+    this.masterServices = masterServices;
+    this.tableNamespaceManager = new TableNamespaceManager(masterServices);
+    this.clusterSchemaOperationTimeoutInMillis = this.masterServices.getConfiguration().
+      getInt(HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY,
+        DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT);
+  }
+
+  // All below are synchronized so consistent view on whether running or not.
+
+  @Override
+  public synchronized boolean isRunning() {
+    return this.running;
+  }
+
+  private synchronized void checkIsRunning() throws ServiceNotRunningException {
+    if (!isRunning()) throw new ServiceNotRunningException();
+  }
+
+  @Override
+  public synchronized void startAndWait() throws IOException {
+    if (isRunning()) throw new IllegalStateException("Already running; cannot double-start.");
+    // Set to running FIRST because tableNamespaceManager start uses this class to do namespace ops
+    this.running = true;
+    this.tableNamespaceManager.start();
+  }
+
+  @Override
+  public synchronized void stopAndWait() throws IOException {
+    checkIsRunning();
+    // You can't stop tableNamespaceManager.
+    this.running = false;
+  }
+
+  @Override
+  public ProcedureInfo get(final Future<ProcedureInfo> future) throws IOException {
+    try {
+      ProcedureInfo pi =
+        future.get(this.clusterSchemaOperationTimeoutInMillis, TimeUnit.MILLISECONDS);
+      // If the procedure got an exception, throw it.
+      if (pi.getException() != null) throw pi.getException();
+      return pi;
+    } catch (ExecutionException ee) {
+      // No cleanup to do... just let the exception out.
+      if (ee.getCause() instanceof IOException) throw (IOException)ee.getCause();
+      else throw new ClusterSchemaException(ee.getCause());
+    } catch (InterruptedException e) {
+      IOException ioe = new InterruptedIOException();
+      ioe.initCause(e);
+      throw ioe;
+    } catch (TimeoutException e) {
+      throw new TimeoutIOException(future.toString());
+    }
+  }
+
+  @Override
+  public TableNamespaceManager getTableNamespaceManager() {
+    return this.tableNamespaceManager;
+  }
+
+  private Future<ProcedureInfo> submitProcedure(final Procedure<?> procedure, long nonceGroup,
+      long nonce)
+  throws ServiceNotRunningException {
+    checkIsRunning();
+    ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
+    long procId = pe.submitProcedure(procedure, nonceGroup, nonce);
+    return new ProcedureFuture(pe, procId);
+  }
+
+  @Override
+  public Future<ProcedureInfo> createNamespace(NamespaceDescriptor namespaceDescriptor,
+      long nonceGroup, long nonce)
+  throws IOException {
+    return submitProcedure(new CreateNamespaceProcedure(
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+        nonceGroup, nonce);
+  }
+
+  @Override
+  public Future<ProcedureInfo> modifyNamespace(NamespaceDescriptor namespaceDescriptor,
+      long nonceGroup, long nonce)
+  throws IOException {
+    return submitProcedure(new ModifyNamespaceProcedure(
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+        nonceGroup, nonce);
+  }
+
+  @Override
+  public Future<ProcedureInfo> deleteNamespace(String name, long nonceGroup, long nonce)
+  throws IOException {
+    return submitProcedure(new DeleteNamespaceProcedure(
+      this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
+        nonceGroup, nonce);
+  }
+
+  @Override
+  public NamespaceDescriptor getNamespace(String name) throws IOException {
+    NamespaceDescriptor nsd = getTableNamespaceManager().get(name);
+    if (nsd == null) throw new NamespaceNotFoundException(name);
+    return nsd;
+  }
+
+  @Override
+  public List<NamespaceDescriptor> getNamespaces() throws IOException {
+    checkIsRunning();
+    Set<NamespaceDescriptor> set = getTableNamespaceManager().list();
+    if (set == null || set.isEmpty()) return EMPTY_NAMESPACE_LIST;
+    List<NamespaceDescriptor> list = new ArrayList<NamespaceDescriptor>(set.size());
+    list.addAll(set);
+    return Collections.unmodifiableList(list);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/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 8e51f25..f29cad6 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
@@ -36,6 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -63,7 +64,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -97,17 +97,14 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
 import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
-import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
-import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
-import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@@ -185,7 +182,7 @@ import com.google.protobuf.Service;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HMaster extends HRegionServer implements MasterServices, Server {
+public class HMaster extends HRegionServer implements MasterServices {
   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
 
   /**
@@ -256,8 +253,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   // Tracker for region normalizer state
   private RegionNormalizerTracker regionNormalizerTracker;
 
-  /** Namespace stuff */
-  private TableNamespaceManager tableNamespaceManager;
+  private ClusterSchemaService clusterSchemaService;
 
   // Metrics for the HMaster
   final MetricsMaster metricsMaster;
@@ -368,9 +364,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * Remaining steps of initialization occur in
    * #finishActiveMasterInitialization(MonitoredTask) after
    * the master becomes the active one.
-   *
-   * @throws KeeperException
-   * @throws IOException
    */
   public HMaster(final Configuration conf, CoordinatedStateManager csm)
       throws IOException, KeeperException {
@@ -570,10 +563,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * Initialize all ZK based system trackers.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws CoordinatedStateException
    */
   void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException, CoordinatedStateException {
@@ -630,11 +619,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * <li>Ensure assignment of meta/namespace regions<li>
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
-   *
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws CoordinatedStateException
    */
   private void finishActiveMasterInitialization(MonitoredTask status)
       throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
@@ -781,8 +765,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     this.catalogJanitorChore = new CatalogJanitor(this, this);
     getChoreService().scheduleChore(catalogJanitorChore);
 
-    status.setStatus("Starting namespace manager");
-    initNamespace();
+    status.setStatus("Starting cluster schema service");
+    initClusterSchemaService();
 
     if (this.cpHost != null) {
       try {
@@ -848,11 +832,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * Create a {@link ServerManager} instance.
-   * @param master
-   * @param services
-   * @return An instance of {@link ServerManager}
-   * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
-   * @throws IOException
    */
   ServerManager createServerManager(final Server master,
       final MasterServices services)
@@ -888,12 +867,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * Check <code>hbase:meta</code> is assigned. If not, assign it.
-   * @param status MonitoredTask
-   * @param previouslyFailedMetaRSs
-   * @param replicaId
-   * @throws InterruptedException
-   * @throws IOException
-   * @throws KeeperException
    */
   void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
       throws InterruptedException, IOException, KeeperException {
@@ -965,10 +938,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     status.setStatus("META assigned.");
   }
 
-  void initNamespace() throws IOException {
-    //create namespace manager
-    tableNamespaceManager = new TableNamespaceManager(this);
-    tableNamespaceManager.start();
+  void initClusterSchemaService() throws IOException, InterruptedException {
+    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
+    this.clusterSchemaService.startAndWait();
+    if (!this.clusterSchemaService.isRunning()) throw new HBaseIOException("Failed start");
   }
 
   void initQuotaManager() throws IOException {
@@ -1014,7 +987,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   /**
    * This function returns a set of region server names under hbase:meta recovering region ZK node
    * @return Set of meta server names which were recorded in ZK
-   * @throws KeeperException
    */
   private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
     Set<ServerName> result = new HashSet<ServerName>();
@@ -1050,11 +1022,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return tableStateManager;
   }
 
-  @Override
-  public TableNamespaceManager getTableNamespaceManager() {
-    return tableNamespaceManager;
-  }
-
   /*
    * Start up all services. If any of these threads gets an unhandled exception
    * then they just die with a logged message.  This should be fine because
@@ -1201,7 +1168,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   /**
    * @return Get remote side's InetAddress
-   * @throws UnknownHostException
    */
   InetAddress getRemoteInetAddress(final int port,
       final long serverStartCode) throws UnknownHostException {
@@ -1338,7 +1304,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    * @return true if normalization step was performed successfully, false otherwise
    *   (specifically, if HMaster hasn't been initialized properly or normalization
    *   is globally disabled)
-   * @throws IOException
    */
   public boolean normalizeRegions() throws IOException {
     if (!this.initialized) {
@@ -1478,9 +1443,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     if (isStopped()) {
       throw new MasterNotRunningException();
     }
-
+    checkInitialized();
     String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
-    ensureNamespaceExists(namespace);
+    this.clusterSchemaService.getNamespace(namespace);
 
     HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
@@ -2305,18 +2270,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
   void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
     checkServiceStarted();
-    if (!this.initialized) {
-      throw new PleaseHoldException("Master is initializing");
-    }
+    if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
   }
 
-  void checkNamespaceManagerReady() throws IOException {
-    checkInitialized();
-    if (tableNamespaceManager == null ||
-        !tableNamespaceManager.isTableAvailableAndInitialized(true)) {
-      throw new IOException("Table Namespace Manager not ready yet, try again later");
-    }
-  }
   /**
    * Report whether this master is currently the active master or not.
    * If not active master, we are parked on ZK waiting to become active.
@@ -2411,7 +2367,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   /**
    * Utility for constructing an instance of the passed HMaster class.
    * @param masterClass
-   * @param conf
    * @return HMaster instance.
    */
   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
@@ -2452,138 +2407,110 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
-  public void createNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
-    checkNamespaceManagerReady();
-    if (cpHost != null) {
-      if (cpHost.preCreateNamespace(descriptor)) {
-        return;
-      }
-    }
-    createNamespaceSync(descriptor, nonceGroup, nonce);
-    if (cpHost != null) {
-      cpHost.postCreateNamespace(descriptor);
-    }
+  public ClusterSchema getClusterSchema() {
+    return this.clusterSchemaService;
   }
 
-  @Override
-  public void createNamespaceSync(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
+  /**
+   * Create a new Namespace.
+   * @param namespaceDescriptor descriptor for new Namespace
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   */
+  void createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
+      final long nonce)
+  throws IOException {
+    checkInitialized();
+    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
+    if (this.cpHost != null && this.cpHost.preCreateNamespace(namespaceDescriptor)) return;
+    LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new CreateNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    Future<ProcedureInfo> future =
+      getClusterSchema().createNamespace(namespaceDescriptor, nonceGroup, nonce);
+    getClusterSchema().get(future);
+    if (this.cpHost != null) this.cpHost.postCreateNamespace(namespaceDescriptor);
   }
 
-  @Override
-  public void modifyNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
-    checkNamespaceManagerReady();
-    if (cpHost != null) {
-      if (cpHost.preModifyNamespace(descriptor)) {
-        return;
-      }
-    }
-    LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
+  /**
+   * Modify an existing Namespace.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).\
+   */
+  void modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
+      final long nonce)
+  throws IOException {
+    checkInitialized();
+    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
+    if (this.cpHost != null && this.cpHost.preModifyNamespace(namespaceDescriptor)) return;
+    LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new ModifyNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
-    if (cpHost != null) {
-      cpHost.postModifyNamespace(descriptor);
-    }
+    Future<ProcedureInfo> future =
+        getClusterSchema().modifyNamespace(namespaceDescriptor, nonceGroup, nonce);
+    getClusterSchema().get(future);
+    if (this.cpHost != null) this.cpHost.postModifyNamespace(namespaceDescriptor);
   }
 
-  @Override
-  public void deleteNamespace(
-      final String name,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    checkNamespaceManagerReady();
-    if (cpHost != null) {
-      if (cpHost.preDeleteNamespace(name)) {
-        return;
-      }
-    }
+  /**
+   * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
+   * @param nonceGroup Identifier for the source of the request, a client or process.
+   * @param nonce A unique identifier for this operation from the client or process identified by
+   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
+   * @throws IOException
+   */
+  void deleteNamespace(final String name, final long nonceGroup, final long nonce)
+  throws IOException {
+    checkInitialized();
+    if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) return;
     LOG.info(getClientIdAuditPrefix() + " delete " + name);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
-    if (cpHost != null) {
-      cpHost.postDeleteNamespace(name);
-    }
+    Future<ProcedureInfo> future = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
+    getClusterSchema().get(future);
+    if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
   }
 
   /**
-   * Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException
-   *
-   * @param name the namespace to check
-   * @throws IOException if the namespace manager is not ready yet.
-   * @throws NamespaceNotFoundException if the namespace does not exists
+   * Get a Namespace
+   * @param name Name of the Namespace
+   * @return Namespace descriptor for <code>name</code>
    */
-  private void ensureNamespaceExists(final String name)
-      throws IOException, NamespaceNotFoundException {
-    checkNamespaceManagerReady();
-    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
-    if (nsd == null) {
-      throw new NamespaceNotFoundException(name);
-    }
+  NamespaceDescriptor getNamespace(String name) throws IOException {
+    checkInitialized();
+    if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
+    NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
+    if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
+    return nsd;
   }
 
-  @Override
-  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
-    checkNamespaceManagerReady();
-
+  /**
+   * Get all Namespaces
+   * @return All Namespace descriptors
+   */
+  List<NamespaceDescriptor> getNamespaces() throws IOException {
+    checkInitialized();
+    final List<NamespaceDescriptor> nsds = new ArrayList<NamespaceDescriptor>();
+    boolean bypass = false;
     if (cpHost != null) {
-      cpHost.preGetNamespaceDescriptor(name);
+      bypass = cpHost.preListNamespaceDescriptors(nsds);
     }
-
-    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
-    if (nsd == null) {
-      throw new NamespaceNotFoundException(name);
-    }
-
-    if (cpHost != null) {
-      cpHost.postGetNamespaceDescriptor(nsd);
+    if (!bypass) {
+      nsds.addAll(this.clusterSchemaService.getNamespaces());
+      if (this.cpHost != null) this.cpHost.postListNamespaceDescriptors(nsds);
     }
-
-    return nsd;
+    return nsds;
   }
 
   @Override
-  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
-    checkNamespaceManagerReady();
-
-    final List<NamespaceDescriptor> descriptors = new ArrayList<NamespaceDescriptor>();
-    boolean bypass = false;
-    if (cpHost != null) {
-      bypass = cpHost.preListNamespaceDescriptors(descriptors);
-    }
-
-    if (!bypass) {
-      descriptors.addAll(tableNamespaceManager.list());
+  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
+    checkInitialized();
+    return listTableNames(name, null, true);
+  }
 
-      if (cpHost != null) {
-        cpHost.postListNamespaceDescriptors(descriptors);
-      }
-    }
-    return descriptors;
+  @Override
+  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+    checkInitialized();
+    return listTableDescriptors(name, null, null, true);
   }
 
   @Override
@@ -2617,18 +2544,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return procInfoList;
   }
 
-  @Override
-  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
-    ensureNamespaceExists(name);
-    return listTableDescriptors(name, null, null, true);
-  }
-
-  @Override
-  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
-    ensureNamespaceExists(name);
-    return listTableNames(name, null, true);
-  }
-
   /**
    * Returns the list of table descriptors that match the specified request
    *
@@ -2653,6 +2568,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         // request for all TableDescriptors
         Collection<HTableDescriptor> htds;
         if (namespace != null && namespace.length() > 0) {
+          // Do a check on the namespace existence. Will fail if does not exist.
+          this.clusterSchemaService.getNamespace(namespace);
           htds = tableDescriptors.getByNamespace(namespace).values();
         } else {
           htds = tableDescriptors.getAll().values();
@@ -2696,46 +2613,17 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    */
   public List<TableName> listTableNames(final String namespace, final String regex,
       final boolean includeSysTables) throws IOException {
-    final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
-
-    boolean bypass = false;
-    if (cpHost != null) {
-      bypass = cpHost.preGetTableNames(descriptors, regex);
-    }
-
+    List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
+    boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): false;
     if (!bypass) {
-      // get all descriptors
-      Collection<HTableDescriptor> htds;
-      if (namespace != null && namespace.length() > 0) {
-        htds = tableDescriptors.getByNamespace(namespace).values();
-      } else {
-        htds = tableDescriptors.getAll().values();
-      }
-
-      for (HTableDescriptor htd: htds) {
-        if (includeSysTables || !htd.getTableName().isSystemTable()) {
-          descriptors.add(htd);
-        }
-      }
-
-      // Retains only those matched by regular expression.
-      if (regex != null) {
-        filterTablesByRegex(descriptors, Pattern.compile(regex));
-      }
-
-      if (cpHost != null) {
-        cpHost.postGetTableNames(descriptors, regex);
-      }
-    }
-
-    List<TableName> result = new ArrayList<TableName>(descriptors.size());
-    for (HTableDescriptor htd: descriptors) {
-      result.add(htd.getTableName());
+      htds.addAll(listTableDescriptors(namespace, regex, null, includeSysTables));
+      if (cpHost != null) cpHost.postGetTableNames(htds, regex);
     }
+    List<TableName> result = new ArrayList<TableName>(htds.size());
+    for (HTableDescriptor htd: htds) result.add(htd.getTableName());
     return result;
   }
 
-
   /**
    * Removes the table descriptors that don't match the pattern.
    * @param descriptors list of table descriptors to filter

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index b269c3d..2528814 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceExistException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerLoad;
@@ -832,7 +833,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       return GetNamespaceDescriptorResponse.newBuilder()
         .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
-            master.getNamespaceDescriptor(request.getNamespaceName())))
+            master.getNamespace(request.getNamespaceName())))
         .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1120,7 +1121,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       ListNamespaceDescriptorsResponse.Builder response =
         ListNamespaceDescriptorsResponse.newBuilder();
-      for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) {
+      for(NamespaceDescriptor ns: master.getNamespaces()) {
         response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
       }
       return response.build();
@@ -1305,10 +1306,9 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       master.snapshotManager.checkSnapshotSupport();
 
-    // ensure namespace exists
+      // Ensure namespace exists. Will throw exception if non-known NS.
       TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
-      master.getNamespaceDescriptor(dstTable.getNamespaceAsString());
-
+      master.getNamespace(dstTable.getNamespaceAsString());
       SnapshotDescription reqSnapshot = request.getSnapshot();
       master.snapshotManager.restoreSnapshot(reqSnapshot);
       return RestoreSnapshotResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index af0e490..ec7db0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,21 +21,20 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 
 import com.google.protobuf.Service;
@@ -46,6 +45,11 @@ import com.google.protobuf.Service;
 @InterfaceAudience.Private
 public interface MasterServices extends Server {
   /**
+   * @return Master's instance of {@link ClusterSchema}
+   */
+  ClusterSchema getClusterSchema();
+
+  /**
    * @return Master's instance of the {@link AssignmentManager}
    */
   AssignmentManager getAssignmentManager();
@@ -81,11 +85,6 @@ public interface MasterServices extends Server {
   MasterCoprocessorHost getMasterCoprocessorHost();
 
   /**
-   * @return Master's instance of {@link TableNamespaceManager}
-   */
-  TableNamespaceManager getTableNamespaceManager();
-
-  /**
    * @return Master's instance of {@link MasterQuotaManager}
    */
   MasterQuotaManager getMasterQuotaManager();
@@ -280,54 +279,6 @@ public interface MasterServices extends Server {
   boolean isInitialized();
 
   /**
-   * Create a new namespace
-   * @param descriptor descriptor which describes the new namespace
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void createNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
-   * Create a new namespace synchronously.
-   * @param descriptor descriptor which describes the new namespace
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void createNamespaceSync(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
-   * Modify an existing namespace
-   * @param descriptor descriptor which updates the existing namespace
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void modifyNamespace(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
-   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
-   * @param name namespace name
-   * @param nonceGroup
-   * @param nonce
-   * @throws IOException
-   */
-  public void deleteNamespace(
-      final String name,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /**
    * Abort a procedure.
    * @param procId ID of the procedure
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
@@ -338,21 +289,6 @@ public interface MasterServices extends Server {
       throws IOException;
 
   /**
-   * Get a namespace descriptor by name
-   * @param name name of namespace descriptor
-   * @return A descriptor
-   * @throws IOException
-   */
-  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException;
-
-  /**
-   * List available namespace descriptors
-   * @return A descriptor
-   * @throws IOException
-   */
-  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException;
-
-  /**
    * List procedures
    * @return procedure list
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
new file mode 100644
index 0000000..0a849fc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
@@ -0,0 +1,132 @@
+/**
+ * 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 java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * Map Future Interface on to Procedure result processing.
+ */
+// Has no extra methods as of now beyond Future<ProcedureInfo>. Use #toString if you want to log
+// procId of procedure.
+// TODO: This should be in Procedure? Have it in master package for now. Lets out ProcedureInfo.
+// Implementation informed by HBaseAdmin#ProcedureFuture.
+@InterfaceAudience.Private
+class ProcedureFuture implements Future<ProcedureInfo> {
+  // Save exception so we can rethrow if called again. Same for result.
+  private ExecutionException exception = null;
+  private ProcedureInfo result = null;
+  private boolean done = false;
+  private boolean cancelled = false;
+  private final Long procId;
+  private final ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
+
+  ProcedureFuture(final ProcedureExecutor<MasterProcedureEnv> procedureExecutor,
+      final long procId) {
+    this.procedureExecutor = procedureExecutor;
+    this.procId = procId;
+  }
+
+  @Override
+  public String toString() {
+    return "procId=" + this.procId;
+  }
+
+  @Override
+  public boolean cancel(boolean mayInterruptIfRunning) {
+    if (!this.cancelled) {
+      this.cancelled = this.procedureExecutor.abort(this.procId, mayInterruptIfRunning);
+    }
+    return this.cancelled;
+  }
+
+  @Override
+  public boolean isCancelled() {
+    return this.cancelled;
+  }
+
+  @Override
+  public boolean isDone() {
+    return done;
+  }
+
+  /**
+   * This method is unsupported. We will throw an UnsupportedOperationException. Only the lazy
+   * would call this method because they can avoid thinking through implication of a Procedure that
+   * might never return so this is disallowed. Use {@link #get(long, TimeUnit)}.
+   */
+  @Override
+  public ProcedureInfo get() throws InterruptedException, ExecutionException {
+    // TODO: should we ever spin forever?
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ProcedureInfo get(long timeout, TimeUnit unit)
+  throws InterruptedException, ExecutionException, TimeoutException {
+    if (!this.done) {
+      // TODO: add this sort of facility to EnvironmentEdgeManager
+      long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
+      try {
+        this.result = waitProcedureResult(procId, deadlineTs);
+      } catch (IOException e) {
+        this.exception = new ExecutionException(e);
+      }
+      this.done = true;
+    }
+    if (exception != null) {
+      throw exception;
+    }
+    return result;
+  }
+
+  /**
+   * @param procId
+   * @param deadlineTs
+   * @return A ProcedureInfo instance or null if procedure not found.
+   * @throws IOException
+   * @throws TimeoutException
+   * @throws InterruptedException
+   */
+  private ProcedureInfo waitProcedureResult(long procId, long deadlineTs)
+  throws IOException, TimeoutException, InterruptedException {
+    while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
+      Pair<ProcedureInfo, Procedure> pair = this.procedureExecutor.getResultOrProcedure(procId);
+      if (pair.getFirst() != null) {
+        this.procedureExecutor.removeResult(procId);
+        return pair.getFirst();
+      } else {
+        if (pair.getSecond() == null) return null;
+      }
+      // TODO: Add a wait.
+    }
+    throw new TimeoutException("The procedure " + procId + " is still running");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index bbeaf76..07822fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -27,17 +27,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.commons.lang.StringUtils;
 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.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZKNamespaceManager;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+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;
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -56,10 +55,12 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import com.google.common.collect.Sets;
 
 /**
- * This is a helper class used to manage the namespace
- * metadata that is stored in TableName.NAMESPACE_TABLE_NAME
- * It also mirrors updates to the ZK store by forwarding updates to
- * {@link org.apache.hadoop.hbase.ZKNamespaceManager}
+ * This is a helper class used internally to manage the namespace metadata that is stored in
+ * TableName.NAMESPACE_TABLE_NAME. It also mirrors updates to the ZK store by forwarding updates to
+ * {@link org.apache.hadoop.hbase.ZKNamespaceManager}.
+ * 
+ * WARNING: Do not use. Go via the higher-level {@link ClusterSchema} API instead. This manager
+ * is likely to go aways anyways.
  */
 @InterfaceAudience.Private
 public class TableNamespaceManager {
@@ -90,7 +91,7 @@ public class TableNamespaceManager {
   private long exclusiveLockTimeoutMs;
   private long sharedLockTimeoutMs;
 
-  public TableNamespaceManager(MasterServices masterServices) {
+  TableNamespaceManager(MasterServices masterServices) {
     this.masterServices = masterServices;
     this.conf = masterServices.getConfiguration();
 
@@ -113,7 +114,7 @@ public class TableNamespaceManager {
       // Wait for the namespace table to be initialized.
       long startTime = EnvironmentEdgeManager.currentTime();
       int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
-      while (!isTableAvailableAndInitialized(false)) {
+      while (!isTableAvailableAndInitialized()) {
         if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
           // We can't do anything if ns is not online.
           throw new IOException("Timedout " + timeout + "ms waiting for namespace table to "
@@ -269,16 +270,29 @@ public class TableNamespaceManager {
   }
 
   /**
+   * Create Namespace in a blocking manner; don't return till success.
+   * Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
+   * @throws IOException 
+   * @throws InterruptedException 
+   */
+  private void createNamespace(final NamespaceDescriptor namespaceDescriptor)
+  throws IOException {
+    ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
+    clusterSchema.get(clusterSchema.
+      createNamespace(namespaceDescriptor, HConstants.NO_NONCE, HConstants.NO_NONCE));
+  }
+
+  /**
    * This method checks if the namespace table is assigned and then
-   * tries to create its HTable. If it was already created before, it also makes
+   * tries to create its Table reference. If it was already created before, it also makes
    * sure that the connection isn't closed.
    * @return true if the namespace table manager is ready to serve, false
    * otherwise
    * @throws IOException
    */
   @SuppressWarnings("deprecation")
-  public synchronized boolean isTableAvailableAndInitialized(
-      final boolean createNamespaceAync) throws IOException {
+  public synchronized boolean isTableAvailableAndInitialized()
+  throws IOException {
     // Did we already get a table? If so, still make sure it's available
     if (isTableNamespaceManagerInitialized()) {
       return true;
@@ -293,34 +307,10 @@ public class TableNamespaceManager {
         zkNamespaceManager.start();
 
         if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
-          if (createNamespaceAync) {
-            masterServices.getMasterProcedureExecutor().submitProcedure(
-              new CreateNamespaceProcedure(
-                masterServices.getMasterProcedureExecutor().getEnvironment(),
-                NamespaceDescriptor.DEFAULT_NAMESPACE));
-            initGoodSofar = false;
-          }
-          else {
-            masterServices.createNamespaceSync(
-              NamespaceDescriptor.DEFAULT_NAMESPACE,
-              HConstants.NO_NONCE,
-              HConstants.NO_NONCE);
-          }
+          createNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
         }
         if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
-          if (createNamespaceAync) {
-            masterServices.getMasterProcedureExecutor().submitProcedure(
-              new CreateNamespaceProcedure(
-                masterServices.getMasterProcedureExecutor().getEnvironment(),
-                NamespaceDescriptor.SYSTEM_NAMESPACE));
-            initGoodSofar = false;
-          }
-          else {
-            masterServices.createNamespaceSync(
-              NamespaceDescriptor.SYSTEM_NAMESPACE,
-              HConstants.NO_NONCE,
-              HConstants.NO_NONCE);
-          }
+          createNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
         }
 
         if (!initGoodSofar) {
@@ -410,4 +400,4 @@ public class TableNamespaceManager {
     }
     return maxRegions;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index 657bbfb..f934737 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -348,7 +348,7 @@ public class CreateNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getTableNamespaceManager();
+    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index 5a42614..2f99167 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -383,7 +383,7 @@ public class DeleteNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getTableNamespaceManager();
+    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
index 30de252..0f8c172 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
@@ -266,8 +266,9 @@ public class ModifyNamespaceProcedure
   }
 
   private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getTableNamespaceManager();
+    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }
+
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
index f24f8c0..8035d32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
@@ -65,7 +65,7 @@ class NamespaceStateManager {
 
   /**
    * Gets an instance of NamespaceTableAndRegionInfo associated with namespace.
-   * @param The name of the namespace
+   * @param name The name of the namespace
    * @return An instance of NamespaceTableAndRegionInfo.
    */
   public NamespaceTableAndRegionInfo getState(String name) {
@@ -135,7 +135,7 @@ class NamespaceStateManager {
 
   private NamespaceDescriptor getNamespaceDescriptor(String namespaceAsString) {
     try {
-      return this.master.getNamespaceDescriptor(namespaceAsString);
+      return this.master.getClusterSchema().getNamespace(namespaceAsString);
     } catch (IOException e) {
       LOG.error("Error while fetching namespace descriptor for namespace : " + namespaceAsString);
       return null;
@@ -212,7 +212,7 @@ class NamespaceStateManager {
    * Initialize namespace state cache by scanning meta table.
    */
   private void initialize() throws IOException {
-    List<NamespaceDescriptor> namespaces = this.master.listNamespaceDescriptors();
+    List<NamespaceDescriptor> namespaces = this.master.getClusterSchema().getNamespaces();
     for (NamespaceDescriptor namespace : namespaces) {
       addNamespace(namespace.getName());
       List<TableName> tables = this.master.listTableNamesByNamespace(namespace.getName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/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 211fed5..1255fa4 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
@@ -196,8 +196,7 @@ import sun.misc.SignalHandler;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HRegionServer extends HasThread implements
-    RegionServerServices, LastSequenceId {
+public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId {
 
   private static final Log LOG = LogFactory.getLog(HRegionServer.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index c24d8a3..f9e2a16 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -358,7 +358,7 @@ public class TestNamespace {
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        admin.listTableDescriptorsByNamespace("non_existing_namespace");
+        admin.listTableDescriptorsByNamespace("non_existant_namespace");
         return null;
       }
     }, NamespaceNotFoundException.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 8e35bbf..785a006 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaMockingUtil;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -143,11 +142,10 @@ public class TestCatalogJanitor {
             ServerName.valueOf("example.org,12345,6789"),
           HRegionInfo.FIRST_META_REGIONINFO);
       // Set hbase.rootdir into test dir.
-      FileSystem fs = FileSystem.get(this.c);
+      FileSystem.get(this.c);
       Path rootdir = FSUtils.getRootDir(this.c);
       FSUtils.setRootDir(this.c, rootdir);
-      AdminProtos.AdminService.BlockingInterface hri =
-        Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
+      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
     }
 
     @Override
@@ -402,48 +400,6 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public void createNamespace(
-        final NamespaceDescriptor descriptor,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public void createNamespaceSync(
-        final NamespaceDescriptor descriptor,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public void modifyNamespace(
-        final NamespaceDescriptor descriptor,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public void deleteNamespace(
-        final String name,
-        final long nonceGroup,
-        final long nonce) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
-      return null;  //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
-    public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
-      return null;  //To change body of implemented methods use File | Settings | File Templates.
-    }
-
-    @Override
     public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
         throws IOException {
       return false;  //To change body of implemented methods use File | Settings | File Templates.
@@ -536,32 +492,29 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public TableNamespaceManager getTableNamespaceManager() {
-      return null;
-    }
-
-    @Override
     public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
         boolean forcible) throws IOException {
     }
 
     @Override
     public boolean isInitialized() {
-      // Auto-generated method stub
       return false;
     }
 
     @Override
     public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
-      // Auto-generated method stub
       return 0;
     }
 
     @Override
     public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
-      // Auto-generated method stub
       return 0;
     }
+
+    @Override
+    public ClusterSchema getClusterSchema() {
+      return null;
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5f3d17a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 972834a..20b492c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -218,10 +218,6 @@ public class TestMasterNoCluster {
           return null;
         }
       }
-
-      @Override
-      void initNamespace() {
-      }
     };
     master.start();
 
@@ -294,10 +290,6 @@ public class TestMasterNoCluster {
           return null;
         }
       }
-
-      @Override
-      void initNamespace() {
-      }
     };
     master.start();
 


[40/50] [abbrv] hbase git commit: HBASE-15065 Addendum fixes checkstyle warnings

Posted by jm...@apache.org.
HBASE-15065 Addendum fixes checkstyle warnings


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

Branch: refs/heads/trunk
Commit: 4a26644245bad6a661442f0eede6850e12d07cc9
Parents: 52280b6
Author: tedyu <yu...@gmail.com>
Authored: Fri Jan 8 14:49:30 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 8 14:49:30 2016 -0800

----------------------------------------------------------------------
 .../hbase/master/normalizer/SimpleRegionNormalizer.java     | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4a266442/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index 7195f7d..7ea6cc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
-import org.apache.hadoop.hbase.util.Triple;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -87,8 +86,12 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
       new Comparator<NormalizationPlan>() {
     @Override
     public int compare(NormalizationPlan plan, NormalizationPlan plan2) {
-      if (plan instanceof SplitNormalizationPlan) return -1;
-      if (plan2 instanceof SplitNormalizationPlan) return 1;
+      if (plan instanceof SplitNormalizationPlan) {
+        return -1;
+      }
+      if (plan2 instanceof SplitNormalizationPlan) {
+        return 1;
+      }
       return 0;
     }
   };


[49/50] [abbrv] hbase git commit: HBASE-15099 Move RegionStateListener class out of quotas package

Posted by jm...@apache.org.
HBASE-15099 Move RegionStateListener class out of quotas package


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

Branch: refs/heads/trunk
Commit: c8b9754a5e0372f93d0e9cf1d9ce788c3d0dcda1
Parents: 5e89ebc
Author: tedyu <yu...@gmail.com>
Authored: Wed Jan 13 15:15:37 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jan 13 15:15:37 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/RegionStateListener.java       | 53 +++++++++++++++++++
 .../hadoop/hbase/master/AssignmentManager.java  |  2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  2 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |  1 +
 .../hbase/quotas/RegionStateListener.java       | 54 --------------------
 5 files changed, 56 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c8b9754a/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
new file mode 100644
index 0000000..22725ec
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * The listener interface for receiving region state events.
+ */
+@InterfaceAudience.Private
+public interface RegionStateListener {
+
+  /**
+   * Process region split event.
+   *
+   * @param hri An instance of HRegionInfo
+   * @throws IOException
+   */
+  void onRegionSplit(HRegionInfo hri) throws IOException;
+
+  /**
+   * Process region split reverted event.
+   *
+   * @param hri An instance of HRegionInfo
+   * @throws IOException Signals that an I/O exception has occurred.
+   */
+  void onRegionSplitReverted(HRegionInfo hri) throws IOException;
+
+  /**
+   * Process region merge event.
+   *
+   * @param hri An instance of HRegionInfo
+   * @throws IOException
+   */
+  void onRegionMerged(HRegionInfo hri) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8b9754a/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 c319bb1..f8132e0 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.RegionStateListener;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
@@ -76,7 +77,6 @@ import org.apache.hadoop.hbase.normalizer.NormalizationPlan.PlanType;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.quotas.QuotaExceededException;
-import org.apache.hadoop.hbase.quotas.RegionStateListener;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8b9754a/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 2431681..8c34b91 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.RegionStateListener;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
@@ -123,7 +124,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRespo
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
-import org.apache.hadoop.hbase.quotas.RegionStateListener;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8b9754a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 8ff633f..caaea67 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.RegionStateListener;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c8b9754a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java
deleted file mode 100644
index 368e21e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionStateListener.java
+++ /dev/null
@@ -1,54 +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.quotas;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * The listener interface for receiving region state events.
- */
-@InterfaceAudience.Private
-public interface RegionStateListener {
-
-  /**
-   * Process region split event.
-   *
-   * @param hri An instance of HRegionInfo
-   * @throws IOException
-   */
-  void onRegionSplit(HRegionInfo hri) throws IOException;
-
-  /**
-   * Process region split reverted event.
-   *
-   * @param hri An instance of HRegionInfo
-   * @throws IOException Signals that an I/O exception has occurred.
-   */
-  void onRegionSplitReverted(HRegionInfo hri) throws IOException;
-
-  /**
-   * Process region merge event.
-   *
-   * @param hri An instance of HRegionInfo
-   * @throws IOException
-   */
-  void onRegionMerged(HRegionInfo hri) throws IOException;
-}


[28/50] [abbrv] hbase git commit: HBASE-12593 Tags to work with ByteBuffer.

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
index 104cb5b..e601af7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
@@ -40,7 +40,9 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -58,6 +60,7 @@ import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.Operator;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -73,7 +76,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
 
   private static final byte[] DUMMY_VALUE = new byte[0];
   private static final byte STRING_SERIALIZATION_FORMAT = 2;
-  private static final Tag STRING_SERIALIZATION_FORMAT_TAG = new Tag(
+  private static final Tag STRING_SERIALIZATION_FORMAT_TAG = new ArrayBackedTag(
       TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE,
       new byte[] { STRING_SERIALIZATION_FORMAT });
   private final ExpressionParser expressionParser = new ExpressionParser();
@@ -281,28 +284,27 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
         boolean visibilityTagPresent = false;
         // Save an object allocation where we can
         if (cell.getTagsLength() > 0) {
-          Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-              cell.getTagsLength());
+          Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell);
           while (tagsItr.hasNext()) {
             boolean includeKV = true;
             Tag tag = tagsItr.next();
             if (tag.getType() == VISIBILITY_TAG_TYPE) {
               visibilityTagPresent = true;
-              int offset = tag.getTagOffset();
-              int endOffset = offset + tag.getTagLength();
+              int offset = tag.getValueOffset();
+              int endOffset = offset + tag.getValueLength();
               while (offset < endOffset) {
-                short len = Bytes.toShort(tag.getBuffer(), offset);
+                short len = getTagValuePartAsShort(tag, offset);
                 offset += 2;
                 if (len < 0) {
                   // This is a NOT label.
                   len = (short) (-1 * len);
-                  String label = Bytes.toString(tag.getBuffer(), offset, len);
+                  String label = Bytes.toString(tag.getValueArray(), offset, len);
                   if (authLabelsFinal.contains(label)) {
                     includeKV = false;
                     break;
                   }
                 } else {
-                  String label = Bytes.toString(tag.getBuffer(), offset, len);
+                  String label = Bytes.toString(tag.getValueArray(), offset, len);
                   if (!authLabelsFinal.contains(label)) {
                     includeKV = false;
                     break;
@@ -353,7 +355,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
       dos.writeShort(bLabel.length);
       dos.write(bLabel);
     }
-    return new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray());
+    return new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray());
   }
 
   private void extractLabels(ExpressionNode node, List<String> labels, List<String> notLabels) {
@@ -423,8 +425,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
       for (Tag tag : deleteVisTags) {
         matchFound = false;
         for (Tag givenTag : putVisTags) {
-          if (Bytes.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(),
-              givenTag.getBuffer(), givenTag.getTagOffset(), givenTag.getTagLength())) {
+          if (TagUtil.matchingValue(tag, givenTag)) {
             matchFound = true;
             break;
           }
@@ -459,15 +460,15 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
           visibilityString.append(VisibilityConstants.CLOSED_PARAN
               + VisibilityConstants.OR_OPERATOR);
         }
-        int offset = tag.getTagOffset();
-        int endOffset = offset + tag.getTagLength();
+        int offset = tag.getValueOffset();
+        int endOffset = offset + tag.getValueLength();
         boolean expressionStart = true;
         while (offset < endOffset) {
-          short len = Bytes.toShort(tag.getBuffer(), offset);
+          short len = getTagValuePartAsShort(tag, offset);
           offset += 2;
           if (len < 0) {
             len = (short) (-1 * len);
-            String label = Bytes.toString(tag.getBuffer(), offset, len);
+            String label = getTagValuePartAsString(tag, offset, len);
             if (expressionStart) {
               visibilityString.append(VisibilityConstants.OPEN_PARAN
                   + VisibilityConstants.NOT_OPERATOR + CellVisibility.quote(label));
@@ -476,7 +477,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
                   + VisibilityConstants.NOT_OPERATOR + CellVisibility.quote(label));
             }
           } else {
-            String label = Bytes.toString(tag.getBuffer(), offset, len);
+            String label = getTagValuePartAsString(tag, offset, len);
             if (expressionStart) {
               visibilityString.append(VisibilityConstants.OPEN_PARAN + CellVisibility.quote(label));
             } else {
@@ -496,4 +497,20 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
     }
     return null;
   }
+
+  private static short getTagValuePartAsShort(Tag t, int offset) {
+    if (t.hasArray()) {
+      return Bytes.toShort(t.getValueArray(), offset);
+    }
+    return ByteBufferUtils.toShort(t.getValueByteBuffer(), offset);
+  }
+
+  private static String getTagValuePartAsString(Tag t, int offset, int length) {
+    if (t.hasArray()) {
+      return Bytes.toString(t.getValueArray(), offset, length);
+    }
+    byte[] b = new byte[length];
+    ByteBufferUtils.copyFromBufferToArray(b, t.getValueByteBuffer(), offset, 0, length);
+    return Bytes.toString(b);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
index fecff07..2140a5c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -183,7 +184,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
           boolean foundNonVisTag = false;
           for(Tag t : TestCoprocessorForTagsAtSink.tags) {
             if(t.getType() == NON_VIS_TAG_TYPE) {
-              assertEquals(TEMP, Bytes.toString(t.getValue()));
+              assertEquals(TEMP, Bytes.toString(TagUtil.cloneValue(t)));
               foundNonVisTag = true;
               break;
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
index 8414813..b3b3b43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java
@@ -43,8 +43,10 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -284,11 +286,11 @@ public class TestVisibilityLabelsReplication {
     for (Cell cell : cells) {
       if ((Bytes.equals(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), row, 0,
           row.length))) {
-        List<Tag> tags = Tag
-            .asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+        List<Tag> tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(),
+            cell.getTagsLength());
         for (Tag tag : tags) {
           if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
-            assertEquals(visTag, Bytes.toString(tag.getValue()));
+            assertEquals(visTag, TagUtil.getValueAsString(tag));
             tagFound = true;
             break;
           }
@@ -330,7 +332,7 @@ public class TestVisibilityLabelsReplication {
           boolean foundNonVisTag = false;
           for (Tag t : TestCoprocessorForTagsAtSink.tags) {
             if (t.getType() == NON_VIS_TAG_TYPE) {
-              assertEquals(TEMP, Bytes.toString(t.getValue()));
+              assertEquals(TEMP, TagUtil.getValueAsString(t));
               foundNonVisTag = true;
               break;
             }
@@ -407,11 +409,11 @@ public class TestVisibilityLabelsReplication {
             if (cf == null) {
               cf = CellUtil.cloneFamily(kv);
             }
-            Tag tag = new Tag((byte) NON_VIS_TAG_TYPE, attribute);
+            Tag tag = new ArrayBackedTag((byte) NON_VIS_TAG_TYPE, attribute);
             List<Tag> tagList = new ArrayList<Tag>();
             tagList.add(tag);
             tagList.addAll(kv.getTags());
-            byte[] fromList = Tag.fromList(tagList);
+            byte[] fromList = TagUtil.fromList(tagList);
             TagRewriteCell newcell = new TagRewriteCell(kv, fromList);
             ((List<Cell>) updatedCells).add(newcell);
           }
@@ -433,7 +435,7 @@ public class TestVisibilityLabelsReplication {
         // Check tag presence in the 1st cell in 1st Result
         if (!results.isEmpty()) {
           Cell cell = results.get(0);
-          tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+          tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
index fdf4fd9..964d6ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
@@ -21,10 +21,13 @@ package org.apache.hadoop.hbase.util;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -98,14 +101,11 @@ public class HFileTestUtil {
         KeyValue kv = new KeyValue(key, family, qualifier, now, key);
         if (withTag) {
           // add a tag.  Arbitrarily chose mob tag since we have a helper already.
-          Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, key);
+          Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, key);
           kv = MobUtils.createMobRefKeyValue(kv, key, tableNameTag);
 
           // verify that the kv has the tag.
-          byte[] ta = kv.getTagsArray();
-          int toff = kv.getTagsOffset();
-          int tlen = kv.getTagsLength();
-          Tag t = Tag.getTag(ta, toff, tlen, TagType.MOB_TABLE_NAME_TAG_TYPE);
+          Tag t = CellUtil.getTag(kv, TagType.MOB_TABLE_NAME_TAG_TYPE);
           if (t == null) {
             throw new IllegalStateException("Tag didn't stick to KV " + kv.toString());
           }
@@ -130,15 +130,12 @@ public class HFileTestUtil {
     ResultScanner s = table.getScanner(new Scan());
     for (Result r : s) {
       for (Cell c : r.listCells()) {
-        byte[] ta = c.getTagsArray();
-        int toff = c.getTagsOffset();
-        int tlen = c.getTagsLength();
-        Tag t = Tag.getTag(ta, toff, tlen, TagType.MOB_TABLE_NAME_TAG_TYPE);
+        Tag t = CellUtil.getTag(c, TagType.MOB_TABLE_NAME_TAG_TYPE);
         if (t == null) {
           fail(c.toString() + " has null tag");
           continue;
         }
-        byte[] tval = t.getValue();
+        byte[] tval = TagUtil.cloneValue(t);
         assertArrayEquals(c.toString() + " has tag" + Bytes.toString(tval),
             r.getRow(), tval);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java
index 70d6d9d..87cb070 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.MultiThreadedAction.DefaultDataGenerator;
@@ -77,7 +78,7 @@ public class LoadTestDataGeneratorWithTags extends DefaultDataGenerator {
             minTagLength + random.nextInt(maxTagLength - minTagLength));
         tags = new ArrayList<Tag>();
         for (int n = 0; n < numTags; n++) {
-          tags.add(new Tag((byte) 127, tag));
+          tags.add(new ArrayBackedTag((byte) 127, tag));
         }
         Cell updatedCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
             cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(),


[39/50] [abbrv] hbase git commit: HBASE-14975 Don't color the total RIT line yellow if it's zero (Pallavi Adusumilli)

Posted by jm...@apache.org.
HBASE-14975 Don't color the total RIT line yellow if it's zero (Pallavi Adusumilli)

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

Branch: refs/heads/trunk
Commit: 52280b64e643584b9b73924a4216442def0501a4
Parents: 6e781a1
Author: Elliott Clark <ec...@apache.org>
Authored: Fri Jan 8 14:29:38 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Fri Jan 8 14:29:44 2016 -0800

----------------------------------------------------------------------
 .../hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon       | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/52280b64/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
index 30a95ce..2bf034a 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
@@ -91,7 +91,12 @@ if (toRemove > 0) {
             entry.getValue(), conf) %></td>
     <td><% (currentTime - entry.getValue().getStamp()) %> </td></tr>
             </%for>
-            <tr BGCOLOR="#D7DF01"> <td>Total number of Regions in Transition for more than <% ritThreshold %> milliseconds</td><td> <% numOfRITOverThreshold %></td><td></td>
+            <%if numOfRITOverThreshold > 0 %>
+                    <tr BGCOLOR="#D7DF01" >
+            <%else>
+                    <tr>
+            </%if>
+            <td>Total number of Regions in Transition for more than <% ritThreshold %> milliseconds</td><td> <% numOfRITOverThreshold %></td><td></td>
             </tr>
     <tr> <td> Total number of Regions in Transition</td><td><% totalRITs %> </td><td></td>
     </table>


[13/50] [abbrv] hbase git commit: Revert "HBASE-14888 ClusterSchema: Add Namespace Operations"

Posted by jm...@apache.org.
Revert "HBASE-14888  ClusterSchema: Add Namespace Operations"

This reverts commit c5f3d17ae3a61cbf77cab89cddd8303e20e5e734.


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

Branch: refs/heads/trunk
Commit: a82f7fc94a52e941b139367f1e67c03229213247
Parents: c5f3d17
Author: stack <st...@apache.org>
Authored: Sun Jan 3 07:15:18 2016 -0800
Committer: stack <st...@apache.org>
Committed: Sun Jan 3 07:15:18 2016 -0800

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  |   2 +-
 .../java/org/apache/hadoop/hbase/Service.java   |  50 ---
 .../hbase/ServiceNotRunningException.java       |  39 ---
 .../apache/hadoop/hbase/ResourceChecker.java    |   2 +-
 .../hadoop/hbase/master/ClusterSchema.java      | 153 ---------
 .../hbase/master/ClusterSchemaException.java    |  37 ---
 .../hbase/master/ClusterSchemaService.java      |  27 --
 .../hbase/master/ClusterSchemaServiceImpl.java  | 164 ----------
 .../org/apache/hadoop/hbase/master/HMaster.java | 314 +++++++++++++------
 .../hadoop/hbase/master/MasterRpcServices.java  |  10 +-
 .../hadoop/hbase/master/MasterServices.java     |  78 ++++-
 .../hadoop/hbase/master/ProcedureFuture.java    | 132 --------
 .../hbase/master/TableNamespaceManager.java     |  68 ++--
 .../procedure/CreateNamespaceProcedure.java     |   2 +-
 .../procedure/DeleteNamespaceProcedure.java     |   2 +-
 .../procedure/ModifyNamespaceProcedure.java     |   3 +-
 .../hbase/namespace/NamespaceStateManager.java  |   6 +-
 .../hbase/regionserver/HRegionServer.java       |   3 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |   2 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |  61 +++-
 .../hbase/master/TestMasterNoCluster.java       |   8 +
 21 files changed, 401 insertions(+), 762 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 3498590..0ef2a17 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -398,7 +398,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       synchronized (this) {
         if (batchPool == null) {
           this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256),
-              conf.getInt("hbase.hconnection.threads.core", 256), "-shared", null);
+              conf.getInt("hbase.hconnection.threads.core", 256), "-shared-", null);
           this.cleanupPool = true;
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
deleted file mode 100644
index 97d93cc..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
+++ /dev/null
@@ -1,50 +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;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Simple Service.
- */
-// This is a WIP. We have Services throughout hbase. Either have all implement what is here or
-// just remove this as an experiment that did not work out.
-// TODO: Move on to guava Service after we update our guava version; later guava has nicer
-// Service implmentation.
-// TODO: Move all Services on to this one Interface.
-@InterfaceAudience.Private
-public interface Service {
-  /**
-   * Initiates service startup (if necessary), returning once the service has finished starting.
-   * @throws IOException Throws exception if already running and if we fail to start successfully.
-   */
-  void startAndWait() throws IOException;
-
-  /**
-   * @return True if this Service is running.
-   */
-  boolean isRunning();
-
-  /**
-   * Initiates service shutdown (if necessary), returning once the service has finished stopping.
-   * @throws IOException Throws exception if not running of if we fail to stop successfully.
-   */
-  void stopAndWait() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
deleted file mode 100644
index f6325ea..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.hadoop.hbase;
-/**
- * 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.
- */
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@SuppressWarnings("serial")
-@InterfaceAudience.Private
-public class ServiceNotRunningException extends HBaseIOException {
-  public ServiceNotRunningException() {
-  }
-
-  public ServiceNotRunningException(String message) {
-    super(message);
-  }
-
-  public ServiceNotRunningException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public ServiceNotRunningException(Throwable cause) {
-    super(cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
index ee0380a..539aea3 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
@@ -41,7 +41,7 @@ public class ResourceChecker {
 
   /**
    * Constructor
-   * @param tagLine The tagLine is added to the logs. Must not be null.
+   * @param tagLine - the tagLine is added to the logs. Must be be null.
    */
   public ResourceChecker(final String tagLine) {
     this.tagLine = tagLine;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
deleted file mode 100644
index e632230..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
+++ /dev/null
@@ -1,153 +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.master;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.List;
-import java.util.concurrent.Future;
-
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * View and edit the current cluster schema. Use this API making any modification to
- * namespaces, tables, etc.
- *
- * <h2>Implementation Notes</h2>
- * Nonces are for when operation is non-idempotent to ensure once-only semantic, even
- * across process failures.
- */
-// ClusterSchema is introduced to encapsulate schema modification. Currently the different aspects
-// are spread about the code base. This effort is about cleanup, shutting down access, and
-// coalescing common code. In particular, we'd contain filesystem modification. Other
-// benefits are to make all schema modification work the same way (one way to do an operation only
-// rather than the current approach where how an operation is done varies with context) and to make
-// it so clusterschema modification can stand apart from Master to faciliate standalone
-// testing. It is part of the filesystem refactor project that undoes the dependency on a
-// layout in HDFS that mimics our model of tables have regions have column families have files.
-// With this Interface in place, with all modifications going via this route where no filesystem
-// particulars are exposed, redoing our internals will take less effort.
-//
-// Currently ClusterSchema Interface will include namespace and table manipulation. Ideally a
-// form of this Interface will go all the ways down to the file manipulation level but currently
-// TBD.
-//
-// ClusterSchema is private to the Master; only the Master knows current cluster state and has
-// means of editing/altering it.
-//
-// TODO: Remove Server argument when MasterServices are passed.
-// TODO: We return Future<ProcedureInfo> in the below from most methods. It may change to return
-// a ProcedureFuture subsequently.
-@InterfaceAudience.Private
-public interface ClusterSchema {
-  /**
-   * Timeout for cluster operations in milliseconds.
-   */
-  public static final String HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY =
-      "hbase.master.cluster.schema.operation.timeout";
-  /**
-   * Default operation timeout in milliseconds.
-   */
-  public static final int DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT =
-      5 * 60 * 1000;
-
-  /**
-   * Utility method that will wait {@link #HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY}
-   * timeout and if exceptions, THROWs the exception doing conversion so palatable outside Master:
-   * i.e. {@link InterruptedException} becomes {@link InterruptedIOException} and so on.
-   *
-   * <<Utility>>
-   *
-   * @param future Future to wait on.
-   * @return On completion, info on the procedure that ran.
-   * @throws IOException
-   */
-  // TODO: Where to put this utility? It goes away?
-  ProcedureInfo get(final Future<ProcedureInfo> future) throws IOException;
-
-  /**
-   * For internals use only. Do not use! Provisionally part of this Interface.
-   * Prefer the high-level APIs available elsewhere in this API.
-   * @return Instance of {@link TableNamespaceManager}
-   */
-  // TODO: Remove from here. Keep internal. This Interface is too high-level to host this accessor.
-  TableNamespaceManager getTableNamespaceManager();
-
-  /**
-   * Create a new Namespace.
-   * @param namespaceDescriptor descriptor for new Namespace
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @return Operation Future.
-   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  Future<ProcedureInfo> createNamespace(NamespaceDescriptor namespaceDescriptor, long nonceGroup,
-      long nonce)
-  throws IOException;
-
-  /**
-   * Modify an existing Namespace.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @return Operation Future.
-   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  Future<ProcedureInfo> modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup,
-      long nonce)
-  throws IOException;
-
-  /**
-   * Delete an existing Namespace.
-   * Only empty Namespaces (no tables) can be removed.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @return Operation Future.
-   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  Future<ProcedureInfo> deleteNamespace(String name, long nonceGroup, long nonce)
-  throws IOException;
-
-  /**
-   * Get a Namespace
-   * @param name Name of the Namespace
-   * @return Namespace descriptor for <code>name</code>
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  // No Future here because presumption is that the request will go against cached metadata so
-  // return immediately -- no need of running a Procedure.
-  NamespaceDescriptor getNamespace(String name) throws IOException;
-
-  /**
-   * Get all Namespaces
-   * @return All Namespace descriptors
-   * @throws IOException
-   */
-  List<NamespaceDescriptor> getNamespaces() throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
deleted file mode 100644
index 62892b6..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
+++ /dev/null
@@ -1,37 +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.master;
-
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@SuppressWarnings("serial")
-@InterfaceAudience.Private
-public class ClusterSchemaException extends HBaseIOException {
-  public ClusterSchemaException(String message) {
-    super(message);
-  }
-
-  public ClusterSchemaException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public ClusterSchemaException(Throwable cause) {
-    super(cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
deleted file mode 100644
index 43353ba..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
+++ /dev/null
@@ -1,27 +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.master;
-
-import org.apache.hadoop.hbase.Service;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Mixes in ClusterSchema and Service
- */
-@InterfaceAudience.Private
-public interface ClusterSchemaService extends ClusterSchema, Service {}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
deleted file mode 100644
index c7f4692..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
+++ /dev/null
@@ -1,164 +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.master;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.ServiceNotRunningException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
-import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
-import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-
-@InterfaceAudience.Private
-class ClusterSchemaServiceImpl implements ClusterSchemaService {
-  private boolean running = false;
-  private final TableNamespaceManager tableNamespaceManager;
-  private final MasterServices masterServices;
-  private final int clusterSchemaOperationTimeoutInMillis;
-  private final static List<NamespaceDescriptor> EMPTY_NAMESPACE_LIST =
-    Collections.unmodifiableList(new ArrayList<NamespaceDescriptor>(0));
-
-  ClusterSchemaServiceImpl(final MasterServices masterServices) {
-    this.masterServices = masterServices;
-    this.tableNamespaceManager = new TableNamespaceManager(masterServices);
-    this.clusterSchemaOperationTimeoutInMillis = this.masterServices.getConfiguration().
-      getInt(HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY,
-        DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT);
-  }
-
-  // All below are synchronized so consistent view on whether running or not.
-
-  @Override
-  public synchronized boolean isRunning() {
-    return this.running;
-  }
-
-  private synchronized void checkIsRunning() throws ServiceNotRunningException {
-    if (!isRunning()) throw new ServiceNotRunningException();
-  }
-
-  @Override
-  public synchronized void startAndWait() throws IOException {
-    if (isRunning()) throw new IllegalStateException("Already running; cannot double-start.");
-    // Set to running FIRST because tableNamespaceManager start uses this class to do namespace ops
-    this.running = true;
-    this.tableNamespaceManager.start();
-  }
-
-  @Override
-  public synchronized void stopAndWait() throws IOException {
-    checkIsRunning();
-    // You can't stop tableNamespaceManager.
-    this.running = false;
-  }
-
-  @Override
-  public ProcedureInfo get(final Future<ProcedureInfo> future) throws IOException {
-    try {
-      ProcedureInfo pi =
-        future.get(this.clusterSchemaOperationTimeoutInMillis, TimeUnit.MILLISECONDS);
-      // If the procedure got an exception, throw it.
-      if (pi.getException() != null) throw pi.getException();
-      return pi;
-    } catch (ExecutionException ee) {
-      // No cleanup to do... just let the exception out.
-      if (ee.getCause() instanceof IOException) throw (IOException)ee.getCause();
-      else throw new ClusterSchemaException(ee.getCause());
-    } catch (InterruptedException e) {
-      IOException ioe = new InterruptedIOException();
-      ioe.initCause(e);
-      throw ioe;
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(future.toString());
-    }
-  }
-
-  @Override
-  public TableNamespaceManager getTableNamespaceManager() {
-    return this.tableNamespaceManager;
-  }
-
-  private Future<ProcedureInfo> submitProcedure(final Procedure<?> procedure, long nonceGroup,
-      long nonce)
-  throws ServiceNotRunningException {
-    checkIsRunning();
-    ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
-    long procId = pe.submitProcedure(procedure, nonceGroup, nonce);
-    return new ProcedureFuture(pe, procId);
-  }
-
-  @Override
-  public Future<ProcedureInfo> createNamespace(NamespaceDescriptor namespaceDescriptor,
-      long nonceGroup, long nonce)
-  throws IOException {
-    return submitProcedure(new CreateNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
-        nonceGroup, nonce);
-  }
-
-  @Override
-  public Future<ProcedureInfo> modifyNamespace(NamespaceDescriptor namespaceDescriptor,
-      long nonceGroup, long nonce)
-  throws IOException {
-    return submitProcedure(new ModifyNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
-        nonceGroup, nonce);
-  }
-
-  @Override
-  public Future<ProcedureInfo> deleteNamespace(String name, long nonceGroup, long nonce)
-  throws IOException {
-    return submitProcedure(new DeleteNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
-        nonceGroup, nonce);
-  }
-
-  @Override
-  public NamespaceDescriptor getNamespace(String name) throws IOException {
-    NamespaceDescriptor nsd = getTableNamespaceManager().get(name);
-    if (nsd == null) throw new NamespaceNotFoundException(name);
-    return nsd;
-  }
-
-  @Override
-  public List<NamespaceDescriptor> getNamespaces() throws IOException {
-    checkIsRunning();
-    Set<NamespaceDescriptor> set = getTableNamespaceManager().list();
-    if (set == null || set.isEmpty()) return EMPTY_NAMESPACE_LIST;
-    List<NamespaceDescriptor> list = new ArrayList<NamespaceDescriptor>(set.size());
-    list.addAll(set);
-    return Collections.unmodifiableList(list);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/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 f29cad6..8e51f25 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
@@ -36,7 +36,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -64,6 +63,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -97,14 +97,17 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
 import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@@ -182,7 +185,7 @@ import com.google.protobuf.Service;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HMaster extends HRegionServer implements MasterServices {
+public class HMaster extends HRegionServer implements MasterServices, Server {
   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
 
   /**
@@ -253,7 +256,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   // Tracker for region normalizer state
   private RegionNormalizerTracker regionNormalizerTracker;
 
-  private ClusterSchemaService clusterSchemaService;
+  /** Namespace stuff */
+  private TableNamespaceManager tableNamespaceManager;
 
   // Metrics for the HMaster
   final MetricsMaster metricsMaster;
@@ -364,6 +368,9 @@ public class HMaster extends HRegionServer implements MasterServices {
    * Remaining steps of initialization occur in
    * #finishActiveMasterInitialization(MonitoredTask) after
    * the master becomes the active one.
+   *
+   * @throws KeeperException
+   * @throws IOException
    */
   public HMaster(final Configuration conf, CoordinatedStateManager csm)
       throws IOException, KeeperException {
@@ -563,6 +570,10 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * Initialize all ZK based system trackers.
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws KeeperException
+   * @throws CoordinatedStateException
    */
   void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException, CoordinatedStateException {
@@ -619,6 +630,11 @@ public class HMaster extends HRegionServer implements MasterServices {
    * <li>Ensure assignment of meta/namespace regions<li>
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
+   *
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws KeeperException
+   * @throws CoordinatedStateException
    */
   private void finishActiveMasterInitialization(MonitoredTask status)
       throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
@@ -765,8 +781,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.catalogJanitorChore = new CatalogJanitor(this, this);
     getChoreService().scheduleChore(catalogJanitorChore);
 
-    status.setStatus("Starting cluster schema service");
-    initClusterSchemaService();
+    status.setStatus("Starting namespace manager");
+    initNamespace();
 
     if (this.cpHost != null) {
       try {
@@ -832,6 +848,11 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * Create a {@link ServerManager} instance.
+   * @param master
+   * @param services
+   * @return An instance of {@link ServerManager}
+   * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
+   * @throws IOException
    */
   ServerManager createServerManager(final Server master,
       final MasterServices services)
@@ -867,6 +888,12 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * Check <code>hbase:meta</code> is assigned. If not, assign it.
+   * @param status MonitoredTask
+   * @param previouslyFailedMetaRSs
+   * @param replicaId
+   * @throws InterruptedException
+   * @throws IOException
+   * @throws KeeperException
    */
   void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
       throws InterruptedException, IOException, KeeperException {
@@ -938,10 +965,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     status.setStatus("META assigned.");
   }
 
-  void initClusterSchemaService() throws IOException, InterruptedException {
-    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
-    this.clusterSchemaService.startAndWait();
-    if (!this.clusterSchemaService.isRunning()) throw new HBaseIOException("Failed start");
+  void initNamespace() throws IOException {
+    //create namespace manager
+    tableNamespaceManager = new TableNamespaceManager(this);
+    tableNamespaceManager.start();
   }
 
   void initQuotaManager() throws IOException {
@@ -987,6 +1014,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * This function returns a set of region server names under hbase:meta recovering region ZK node
    * @return Set of meta server names which were recorded in ZK
+   * @throws KeeperException
    */
   private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
     Set<ServerName> result = new HashSet<ServerName>();
@@ -1022,6 +1050,11 @@ public class HMaster extends HRegionServer implements MasterServices {
     return tableStateManager;
   }
 
+  @Override
+  public TableNamespaceManager getTableNamespaceManager() {
+    return tableNamespaceManager;
+  }
+
   /*
    * Start up all services. If any of these threads gets an unhandled exception
    * then they just die with a logged message.  This should be fine because
@@ -1168,6 +1201,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * @return Get remote side's InetAddress
+   * @throws UnknownHostException
    */
   InetAddress getRemoteInetAddress(final int port,
       final long serverStartCode) throws UnknownHostException {
@@ -1304,6 +1338,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @return true if normalization step was performed successfully, false otherwise
    *   (specifically, if HMaster hasn't been initialized properly or normalization
    *   is globally disabled)
+   * @throws IOException
    */
   public boolean normalizeRegions() throws IOException {
     if (!this.initialized) {
@@ -1443,9 +1478,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (isStopped()) {
       throw new MasterNotRunningException();
     }
-    checkInitialized();
+
     String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
-    this.clusterSchemaService.getNamespace(namespace);
+    ensureNamespaceExists(namespace);
 
     HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
@@ -2270,9 +2305,18 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
     checkServiceStarted();
-    if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
+    if (!this.initialized) {
+      throw new PleaseHoldException("Master is initializing");
+    }
   }
 
+  void checkNamespaceManagerReady() throws IOException {
+    checkInitialized();
+    if (tableNamespaceManager == null ||
+        !tableNamespaceManager.isTableAvailableAndInitialized(true)) {
+      throw new IOException("Table Namespace Manager not ready yet, try again later");
+    }
+  }
   /**
    * Report whether this master is currently the active master or not.
    * If not active master, we are parked on ZK waiting to become active.
@@ -2367,6 +2411,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * Utility for constructing an instance of the passed HMaster class.
    * @param masterClass
+   * @param conf
    * @return HMaster instance.
    */
   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
@@ -2407,110 +2452,138 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ClusterSchema getClusterSchema() {
-    return this.clusterSchemaService;
+  public void createNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
+    checkNamespaceManagerReady();
+    if (cpHost != null) {
+      if (cpHost.preCreateNamespace(descriptor)) {
+        return;
+      }
+    }
+    createNamespaceSync(descriptor, nonceGroup, nonce);
+    if (cpHost != null) {
+      cpHost.postCreateNamespace(descriptor);
+    }
   }
 
-  /**
-   * Create a new Namespace.
-   * @param namespaceDescriptor descriptor for new Namespace
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   */
-  void createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
-      final long nonce)
-  throws IOException {
-    checkInitialized();
-    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
-    if (this.cpHost != null && this.cpHost.preCreateNamespace(namespaceDescriptor)) return;
-    LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
+  @Override
+  public void createNamespaceSync(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    Future<ProcedureInfo> future =
-      getClusterSchema().createNamespace(namespaceDescriptor, nonceGroup, nonce);
-    getClusterSchema().get(future);
-    if (this.cpHost != null) this.cpHost.postCreateNamespace(namespaceDescriptor);
+    long procId = this.procedureExecutor.submitProcedure(
+      new CreateNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
+      nonceGroup,
+      nonce);
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
   }
 
-  /**
-   * Modify an existing Namespace.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).\
-   */
-  void modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
-      final long nonce)
-  throws IOException {
-    checkInitialized();
-    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
-    if (this.cpHost != null && this.cpHost.preModifyNamespace(namespaceDescriptor)) return;
-    LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
+  @Override
+  public void modifyNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
+    checkNamespaceManagerReady();
+    if (cpHost != null) {
+      if (cpHost.preModifyNamespace(descriptor)) {
+        return;
+      }
+    }
+    LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    Future<ProcedureInfo> future =
-        getClusterSchema().modifyNamespace(namespaceDescriptor, nonceGroup, nonce);
-    getClusterSchema().get(future);
-    if (this.cpHost != null) this.cpHost.postModifyNamespace(namespaceDescriptor);
+    long procId = this.procedureExecutor.submitProcedure(
+      new ModifyNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
+      nonceGroup,
+      nonce);
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    if (cpHost != null) {
+      cpHost.postModifyNamespace(descriptor);
+    }
   }
 
-  /**
-   * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @throws IOException
-   */
-  void deleteNamespace(final String name, final long nonceGroup, final long nonce)
-  throws IOException {
-    checkInitialized();
-    if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) return;
+  @Override
+  public void deleteNamespace(
+      final String name,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    checkNamespaceManagerReady();
+    if (cpHost != null) {
+      if (cpHost.preDeleteNamespace(name)) {
+        return;
+      }
+    }
     LOG.info(getClientIdAuditPrefix() + " delete " + name);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    Future<ProcedureInfo> future = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
-    getClusterSchema().get(future);
-    if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
+    long procId = this.procedureExecutor.submitProcedure(
+      new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name),
+      nonceGroup,
+      nonce);
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    if (cpHost != null) {
+      cpHost.postDeleteNamespace(name);
+    }
   }
 
   /**
-   * Get a Namespace
-   * @param name Name of the Namespace
-   * @return Namespace descriptor for <code>name</code>
+   * Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException
+   *
+   * @param name the namespace to check
+   * @throws IOException if the namespace manager is not ready yet.
+   * @throws NamespaceNotFoundException if the namespace does not exists
    */
-  NamespaceDescriptor getNamespace(String name) throws IOException {
-    checkInitialized();
-    if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
-    NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
-    if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
-    return nsd;
+  private void ensureNamespaceExists(final String name)
+      throws IOException, NamespaceNotFoundException {
+    checkNamespaceManagerReady();
+    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
+    if (nsd == null) {
+      throw new NamespaceNotFoundException(name);
+    }
   }
 
-  /**
-   * Get all Namespaces
-   * @return All Namespace descriptors
-   */
-  List<NamespaceDescriptor> getNamespaces() throws IOException {
-    checkInitialized();
-    final List<NamespaceDescriptor> nsds = new ArrayList<NamespaceDescriptor>();
-    boolean bypass = false;
+  @Override
+  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
+    checkNamespaceManagerReady();
+
     if (cpHost != null) {
-      bypass = cpHost.preListNamespaceDescriptors(nsds);
+      cpHost.preGetNamespaceDescriptor(name);
     }
-    if (!bypass) {
-      nsds.addAll(this.clusterSchemaService.getNamespaces());
-      if (this.cpHost != null) this.cpHost.postListNamespaceDescriptors(nsds);
+
+    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
+    if (nsd == null) {
+      throw new NamespaceNotFoundException(name);
     }
-    return nsds;
-  }
 
-  @Override
-  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
-    checkInitialized();
-    return listTableNames(name, null, true);
+    if (cpHost != null) {
+      cpHost.postGetNamespaceDescriptor(nsd);
+    }
+
+    return nsd;
   }
 
   @Override
-  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
-    checkInitialized();
-    return listTableDescriptors(name, null, null, true);
+  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
+    checkNamespaceManagerReady();
+
+    final List<NamespaceDescriptor> descriptors = new ArrayList<NamespaceDescriptor>();
+    boolean bypass = false;
+    if (cpHost != null) {
+      bypass = cpHost.preListNamespaceDescriptors(descriptors);
+    }
+
+    if (!bypass) {
+      descriptors.addAll(tableNamespaceManager.list());
+
+      if (cpHost != null) {
+        cpHost.postListNamespaceDescriptors(descriptors);
+      }
+    }
+    return descriptors;
   }
 
   @Override
@@ -2544,6 +2617,18 @@ public class HMaster extends HRegionServer implements MasterServices {
     return procInfoList;
   }
 
+  @Override
+  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+    ensureNamespaceExists(name);
+    return listTableDescriptors(name, null, null, true);
+  }
+
+  @Override
+  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
+    ensureNamespaceExists(name);
+    return listTableNames(name, null, true);
+  }
+
   /**
    * Returns the list of table descriptors that match the specified request
    *
@@ -2568,8 +2653,6 @@ public class HMaster extends HRegionServer implements MasterServices {
         // request for all TableDescriptors
         Collection<HTableDescriptor> htds;
         if (namespace != null && namespace.length() > 0) {
-          // Do a check on the namespace existence. Will fail if does not exist.
-          this.clusterSchemaService.getNamespace(namespace);
           htds = tableDescriptors.getByNamespace(namespace).values();
         } else {
           htds = tableDescriptors.getAll().values();
@@ -2613,17 +2696,46 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   public List<TableName> listTableNames(final String namespace, final String regex,
       final boolean includeSysTables) throws IOException {
-    List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
-    boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): false;
+    final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
+
+    boolean bypass = false;
+    if (cpHost != null) {
+      bypass = cpHost.preGetTableNames(descriptors, regex);
+    }
+
     if (!bypass) {
-      htds.addAll(listTableDescriptors(namespace, regex, null, includeSysTables));
-      if (cpHost != null) cpHost.postGetTableNames(htds, regex);
+      // get all descriptors
+      Collection<HTableDescriptor> htds;
+      if (namespace != null && namespace.length() > 0) {
+        htds = tableDescriptors.getByNamespace(namespace).values();
+      } else {
+        htds = tableDescriptors.getAll().values();
+      }
+
+      for (HTableDescriptor htd: htds) {
+        if (includeSysTables || !htd.getTableName().isSystemTable()) {
+          descriptors.add(htd);
+        }
+      }
+
+      // Retains only those matched by regular expression.
+      if (regex != null) {
+        filterTablesByRegex(descriptors, Pattern.compile(regex));
+      }
+
+      if (cpHost != null) {
+        cpHost.postGetTableNames(descriptors, regex);
+      }
+    }
+
+    List<TableName> result = new ArrayList<TableName>(descriptors.size());
+    for (HTableDescriptor htd: descriptors) {
+      result.add(htd.getTableName());
     }
-    List<TableName> result = new ArrayList<TableName>(htds.size());
-    for (HTableDescriptor htd: htds) result.add(htd.getTableName());
     return result;
   }
 
+
   /**
    * Removes the table descriptors that don't match the pattern.
    * @param descriptors list of table descriptors to filter

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 2528814..b269c3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceExistException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerLoad;
@@ -833,7 +832,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       return GetNamespaceDescriptorResponse.newBuilder()
         .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
-            master.getNamespace(request.getNamespaceName())))
+            master.getNamespaceDescriptor(request.getNamespaceName())))
         .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1121,7 +1120,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       ListNamespaceDescriptorsResponse.Builder response =
         ListNamespaceDescriptorsResponse.newBuilder();
-      for(NamespaceDescriptor ns: master.getNamespaces()) {
+      for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) {
         response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
       }
       return response.build();
@@ -1306,9 +1305,10 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       master.snapshotManager.checkSnapshotSupport();
 
-      // Ensure namespace exists. Will throw exception if non-known NS.
+    // ensure namespace exists
       TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
-      master.getNamespace(dstTable.getNamespaceAsString());
+      master.getNamespaceDescriptor(dstTable.getNamespaceAsString());
+
       SnapshotDescription reqSnapshot = request.getSnapshot();
       master.snapshotManager.restoreSnapshot(reqSnapshot);
       return RestoreSnapshotResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index ec7db0c..af0e490 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,20 +21,21 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 
 import com.google.protobuf.Service;
@@ -45,11 +46,6 @@ import com.google.protobuf.Service;
 @InterfaceAudience.Private
 public interface MasterServices extends Server {
   /**
-   * @return Master's instance of {@link ClusterSchema}
-   */
-  ClusterSchema getClusterSchema();
-
-  /**
    * @return Master's instance of the {@link AssignmentManager}
    */
   AssignmentManager getAssignmentManager();
@@ -85,6 +81,11 @@ public interface MasterServices extends Server {
   MasterCoprocessorHost getMasterCoprocessorHost();
 
   /**
+   * @return Master's instance of {@link TableNamespaceManager}
+   */
+  TableNamespaceManager getTableNamespaceManager();
+
+  /**
    * @return Master's instance of {@link MasterQuotaManager}
    */
   MasterQuotaManager getMasterQuotaManager();
@@ -279,6 +280,54 @@ public interface MasterServices extends Server {
   boolean isInitialized();
 
   /**
+   * Create a new namespace
+   * @param descriptor descriptor which describes the new namespace
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void createNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
+   * Create a new namespace synchronously.
+   * @param descriptor descriptor which describes the new namespace
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void createNamespaceSync(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
+   * Modify an existing namespace
+   * @param descriptor descriptor which updates the existing namespace
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void modifyNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
+   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+   * @param name namespace name
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void deleteNamespace(
+      final String name,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
    * Abort a procedure.
    * @param procId ID of the procedure
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
@@ -289,6 +338,21 @@ public interface MasterServices extends Server {
       throws IOException;
 
   /**
+   * Get a namespace descriptor by name
+   * @param name name of namespace descriptor
+   * @return A descriptor
+   * @throws IOException
+   */
+  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException;
+
+  /**
+   * List available namespace descriptors
+   * @return A descriptor
+   * @throws IOException
+   */
+  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException;
+
+  /**
    * List procedures
    * @return procedure list
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
deleted file mode 100644
index 0a849fc..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
+++ /dev/null
@@ -1,132 +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.master;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * Map Future Interface on to Procedure result processing.
- */
-// Has no extra methods as of now beyond Future<ProcedureInfo>. Use #toString if you want to log
-// procId of procedure.
-// TODO: This should be in Procedure? Have it in master package for now. Lets out ProcedureInfo.
-// Implementation informed by HBaseAdmin#ProcedureFuture.
-@InterfaceAudience.Private
-class ProcedureFuture implements Future<ProcedureInfo> {
-  // Save exception so we can rethrow if called again. Same for result.
-  private ExecutionException exception = null;
-  private ProcedureInfo result = null;
-  private boolean done = false;
-  private boolean cancelled = false;
-  private final Long procId;
-  private final ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
-
-  ProcedureFuture(final ProcedureExecutor<MasterProcedureEnv> procedureExecutor,
-      final long procId) {
-    this.procedureExecutor = procedureExecutor;
-    this.procId = procId;
-  }
-
-  @Override
-  public String toString() {
-    return "procId=" + this.procId;
-  }
-
-  @Override
-  public boolean cancel(boolean mayInterruptIfRunning) {
-    if (!this.cancelled) {
-      this.cancelled = this.procedureExecutor.abort(this.procId, mayInterruptIfRunning);
-    }
-    return this.cancelled;
-  }
-
-  @Override
-  public boolean isCancelled() {
-    return this.cancelled;
-  }
-
-  @Override
-  public boolean isDone() {
-    return done;
-  }
-
-  /**
-   * This method is unsupported. We will throw an UnsupportedOperationException. Only the lazy
-   * would call this method because they can avoid thinking through implication of a Procedure that
-   * might never return so this is disallowed. Use {@link #get(long, TimeUnit)}.
-   */
-  @Override
-  public ProcedureInfo get() throws InterruptedException, ExecutionException {
-    // TODO: should we ever spin forever?
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ProcedureInfo get(long timeout, TimeUnit unit)
-  throws InterruptedException, ExecutionException, TimeoutException {
-    if (!this.done) {
-      // TODO: add this sort of facility to EnvironmentEdgeManager
-      long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
-      try {
-        this.result = waitProcedureResult(procId, deadlineTs);
-      } catch (IOException e) {
-        this.exception = new ExecutionException(e);
-      }
-      this.done = true;
-    }
-    if (exception != null) {
-      throw exception;
-    }
-    return result;
-  }
-
-  /**
-   * @param procId
-   * @param deadlineTs
-   * @return A ProcedureInfo instance or null if procedure not found.
-   * @throws IOException
-   * @throws TimeoutException
-   * @throws InterruptedException
-   */
-  private ProcedureInfo waitProcedureResult(long procId, long deadlineTs)
-  throws IOException, TimeoutException, InterruptedException {
-    while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
-      Pair<ProcedureInfo, Procedure> pair = this.procedureExecutor.getResultOrProcedure(procId);
-      if (pair.getFirst() != null) {
-        this.procedureExecutor.removeResult(procId);
-        return pair.getFirst();
-      } else {
-        if (pair.getSecond() == null) return null;
-      }
-      // TODO: Add a wait.
-    }
-    throw new TimeoutException("The procedure " + procId + " is still running");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index 07822fd..bbeaf76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -27,17 +27,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.commons.lang.StringUtils;
 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.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZKNamespaceManager;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -55,12 +56,10 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import com.google.common.collect.Sets;
 
 /**
- * This is a helper class used internally to manage the namespace metadata that is stored in
- * TableName.NAMESPACE_TABLE_NAME. It also mirrors updates to the ZK store by forwarding updates to
- * {@link org.apache.hadoop.hbase.ZKNamespaceManager}.
- * 
- * WARNING: Do not use. Go via the higher-level {@link ClusterSchema} API instead. This manager
- * is likely to go aways anyways.
+ * This is a helper class used to manage the namespace
+ * metadata that is stored in TableName.NAMESPACE_TABLE_NAME
+ * It also mirrors updates to the ZK store by forwarding updates to
+ * {@link org.apache.hadoop.hbase.ZKNamespaceManager}
  */
 @InterfaceAudience.Private
 public class TableNamespaceManager {
@@ -91,7 +90,7 @@ public class TableNamespaceManager {
   private long exclusiveLockTimeoutMs;
   private long sharedLockTimeoutMs;
 
-  TableNamespaceManager(MasterServices masterServices) {
+  public TableNamespaceManager(MasterServices masterServices) {
     this.masterServices = masterServices;
     this.conf = masterServices.getConfiguration();
 
@@ -114,7 +113,7 @@ public class TableNamespaceManager {
       // Wait for the namespace table to be initialized.
       long startTime = EnvironmentEdgeManager.currentTime();
       int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
-      while (!isTableAvailableAndInitialized()) {
+      while (!isTableAvailableAndInitialized(false)) {
         if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
           // We can't do anything if ns is not online.
           throw new IOException("Timedout " + timeout + "ms waiting for namespace table to "
@@ -270,29 +269,16 @@ public class TableNamespaceManager {
   }
 
   /**
-   * Create Namespace in a blocking manner; don't return till success.
-   * Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
-   * @throws IOException 
-   * @throws InterruptedException 
-   */
-  private void createNamespace(final NamespaceDescriptor namespaceDescriptor)
-  throws IOException {
-    ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
-    clusterSchema.get(clusterSchema.
-      createNamespace(namespaceDescriptor, HConstants.NO_NONCE, HConstants.NO_NONCE));
-  }
-
-  /**
    * This method checks if the namespace table is assigned and then
-   * tries to create its Table reference. If it was already created before, it also makes
+   * tries to create its HTable. If it was already created before, it also makes
    * sure that the connection isn't closed.
    * @return true if the namespace table manager is ready to serve, false
    * otherwise
    * @throws IOException
    */
   @SuppressWarnings("deprecation")
-  public synchronized boolean isTableAvailableAndInitialized()
-  throws IOException {
+  public synchronized boolean isTableAvailableAndInitialized(
+      final boolean createNamespaceAync) throws IOException {
     // Did we already get a table? If so, still make sure it's available
     if (isTableNamespaceManagerInitialized()) {
       return true;
@@ -307,10 +293,34 @@ public class TableNamespaceManager {
         zkNamespaceManager.start();
 
         if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
-          createNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
+          if (createNamespaceAync) {
+            masterServices.getMasterProcedureExecutor().submitProcedure(
+              new CreateNamespaceProcedure(
+                masterServices.getMasterProcedureExecutor().getEnvironment(),
+                NamespaceDescriptor.DEFAULT_NAMESPACE));
+            initGoodSofar = false;
+          }
+          else {
+            masterServices.createNamespaceSync(
+              NamespaceDescriptor.DEFAULT_NAMESPACE,
+              HConstants.NO_NONCE,
+              HConstants.NO_NONCE);
+          }
         }
         if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
-          createNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
+          if (createNamespaceAync) {
+            masterServices.getMasterProcedureExecutor().submitProcedure(
+              new CreateNamespaceProcedure(
+                masterServices.getMasterProcedureExecutor().getEnvironment(),
+                NamespaceDescriptor.SYSTEM_NAMESPACE));
+            initGoodSofar = false;
+          }
+          else {
+            masterServices.createNamespaceSync(
+              NamespaceDescriptor.SYSTEM_NAMESPACE,
+              HConstants.NO_NONCE,
+              HConstants.NO_NONCE);
+          }
         }
 
         if (!initGoodSofar) {
@@ -400,4 +410,4 @@ public class TableNamespaceManager {
     }
     return maxRegions;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index f934737..657bbfb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -348,7 +348,7 @@ public class CreateNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
+    return env.getMasterServices().getTableNamespaceManager();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index 2f99167..5a42614 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -383,7 +383,7 @@ public class DeleteNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
+    return env.getMasterServices().getTableNamespaceManager();
   }
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
index 0f8c172..30de252 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
@@ -266,9 +266,8 @@ public class ModifyNamespaceProcedure
   }
 
   private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
+    return env.getMasterServices().getTableNamespaceManager();
   }
-
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
index 8035d32..f24f8c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
@@ -65,7 +65,7 @@ class NamespaceStateManager {
 
   /**
    * Gets an instance of NamespaceTableAndRegionInfo associated with namespace.
-   * @param name The name of the namespace
+   * @param The name of the namespace
    * @return An instance of NamespaceTableAndRegionInfo.
    */
   public NamespaceTableAndRegionInfo getState(String name) {
@@ -135,7 +135,7 @@ class NamespaceStateManager {
 
   private NamespaceDescriptor getNamespaceDescriptor(String namespaceAsString) {
     try {
-      return this.master.getClusterSchema().getNamespace(namespaceAsString);
+      return this.master.getNamespaceDescriptor(namespaceAsString);
     } catch (IOException e) {
       LOG.error("Error while fetching namespace descriptor for namespace : " + namespaceAsString);
       return null;
@@ -212,7 +212,7 @@ class NamespaceStateManager {
    * Initialize namespace state cache by scanning meta table.
    */
   private void initialize() throws IOException {
-    List<NamespaceDescriptor> namespaces = this.master.getClusterSchema().getNamespaces();
+    List<NamespaceDescriptor> namespaces = this.master.listNamespaceDescriptors();
     for (NamespaceDescriptor namespace : namespaces) {
       addNamespace(namespace.getName());
       List<TableName> tables = this.master.listTableNamesByNamespace(namespace.getName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/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 1255fa4..211fed5 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
@@ -196,7 +196,8 @@ import sun.misc.SignalHandler;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId {
+public class HRegionServer extends HasThread implements
+    RegionServerServices, LastSequenceId {
 
   private static final Log LOG = LogFactory.getLog(HRegionServer.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index f9e2a16..c24d8a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -358,7 +358,7 @@ public class TestNamespace {
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        admin.listTableDescriptorsByNamespace("non_existant_namespace");
+        admin.listTableDescriptorsByNamespace("non_existing_namespace");
         return null;
       }
     }, NamespaceNotFoundException.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 785a006..8e35bbf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaMockingUtil;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -142,10 +143,11 @@ public class TestCatalogJanitor {
             ServerName.valueOf("example.org,12345,6789"),
           HRegionInfo.FIRST_META_REGIONINFO);
       // Set hbase.rootdir into test dir.
-      FileSystem.get(this.c);
+      FileSystem fs = FileSystem.get(this.c);
       Path rootdir = FSUtils.getRootDir(this.c);
       FSUtils.setRootDir(this.c, rootdir);
-      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
+      AdminProtos.AdminService.BlockingInterface hri =
+        Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
     }
 
     @Override
@@ -400,6 +402,48 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public void createNamespace(
+        final NamespaceDescriptor descriptor,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public void createNamespaceSync(
+        final NamespaceDescriptor descriptor,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public void modifyNamespace(
+        final NamespaceDescriptor descriptor,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public void deleteNamespace(
+        final String name,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
     public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
         throws IOException {
       return false;  //To change body of implemented methods use File | Settings | File Templates.
@@ -492,29 +536,32 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public TableNamespaceManager getTableNamespaceManager() {
+      return null;
+    }
+
+    @Override
     public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
         boolean forcible) throws IOException {
     }
 
     @Override
     public boolean isInitialized() {
+      // Auto-generated method stub
       return false;
     }
 
     @Override
     public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
+      // Auto-generated method stub
       return 0;
     }
 
     @Override
     public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
+      // Auto-generated method stub
       return 0;
     }
-
-    @Override
-    public ClusterSchema getClusterSchema() {
-      return null;
-    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 20b492c..972834a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -218,6 +218,10 @@ public class TestMasterNoCluster {
           return null;
         }
       }
+
+      @Override
+      void initNamespace() {
+      }
     };
     master.start();
 
@@ -290,6 +294,10 @@ public class TestMasterNoCluster {
           return null;
         }
       }
+
+      @Override
+      void initNamespace() {
+      }
     };
     master.start();
 


[08/50] [abbrv] hbase git commit: HBASE-15050 Block Ref counting does not work in Region Split cases (Ram)

Posted by jm...@apache.org.
HBASE-15050 Block Ref counting does not work in Region Split cases (Ram)


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

Branch: refs/heads/trunk
Commit: 07b623670647686084f8f5fd2038e2bafcfdac54
Parents: 413d663
Author: ramkrishna <ra...@gmail.com>
Authored: Wed Dec 30 14:39:03 2015 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Wed Dec 30 14:39:03 2015 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/io/HalfStoreFileReader.java    |  8 +++
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |  8 ++-
 .../client/TestBlockEvictionFromClient.java     | 53 ++++++++++++++++++++
 3 files changed, 68 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/07b62367/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
index 0dd7742..067d24c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
@@ -314,6 +314,10 @@ public class HalfStoreFileReader extends StoreFile.Reader {
       }
     } catch (IOException e) {
       LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
     }
     return null;
   }
@@ -335,6 +339,10 @@ public class HalfStoreFileReader extends StoreFile.Reader {
         firstKeySeeked = true;
       } catch (IOException e) {
         LOG.warn("Failed seekTo first KV in the file", e);
+      } finally {
+        if(scanner != null) {
+          scanner.close();
+        }
       }
     }
     return this.firstKey;

http://git-wip-us.apache.org/repos/asf/hbase/blob/07b62367/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 4e2ca7d..fcf7b5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -986,7 +986,13 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         return new KeyValue.KeyOnlyKeyValue(keyBuf.array(), keyBuf.arrayOffset()
             + keyPair.getSecond(), currKeyLen);
       } else {
-        return new ByteBufferedKeyOnlyKeyValue(keyBuf, keyPair.getSecond(), currKeyLen);
+        // Better to do a copy here instead of holding on to this BB so that
+        // we could release the blocks referring to this key. This key is specifically used 
+        // in HalfStoreFileReader to get the firstkey and lastkey by creating a new scanner
+        // every time. So holding onto the BB (incase of DBB) is not advised here.
+        byte[] key = new byte[currKeyLen];
+        ByteBufferUtils.copyFromBufferToArray(key, keyBuf, keyPair.getSecond(), 0, currKeyLen);
+        return new KeyValue.KeyOnlyKeyValue(key, 0, currKeyLen);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/07b62367/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index a812623..f4d668c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -74,6 +74,8 @@ public class TestBlockEvictionFromClient {
   private static int NO_OF_THREADS = 3;
   private static byte[] ROW = Bytes.toBytes("testRow");
   private static byte[] ROW1 = Bytes.toBytes("testRow1");
+  private static byte[] ROW2 = Bytes.toBytes("testRow2");
+  private static byte[] ROW3 = Bytes.toBytes("testRow3");
   private static byte[] FAMILY = Bytes.toBytes("testFamily");
   private static byte[][] FAMILIES_1 = new byte[1][0];
   private static byte[] QUALIFIER = Bytes.toBytes("testQualifier");
@@ -554,6 +556,57 @@ public class TestBlockEvictionFromClient {
   }
 
   @Test
+  public void testBlockRefCountAfterSplits() throws IOException, InterruptedException {
+    HTable table = null;
+    try {
+      TableName tableName = TableName.valueOf("testBlockRefCountAfterSplits");
+      table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024);
+      // get the block cache and region
+      RegionLocator locator = table.getRegionLocator();
+      String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
+      Region region =
+          TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
+      Store store = region.getStores().iterator().next();
+      CacheConfig cacheConf = store.getCacheConfig();
+      cacheConf.setEvictOnClose(true);
+      BlockCache cache = cacheConf.getBlockCache();
+
+      Put put = new Put(ROW);
+      put.addColumn(FAMILY, QUALIFIER, data);
+      table.put(put);
+      region.flush(true);
+      put = new Put(ROW1);
+      put.addColumn(FAMILY, QUALIFIER, data);
+      table.put(put);
+      region.flush(true);
+      byte[] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
+      put = new Put(ROW2);
+      put.addColumn(FAMILY, QUALIFIER2, data2);
+      table.put(put);
+      put = new Put(ROW3);
+      put.addColumn(FAMILY, QUALIFIER2, data2);
+      table.put(put);
+      region.flush(true);
+      TEST_UTIL.getAdmin().split(tableName, ROW1);
+      List<HRegionInfo> tableRegions = TEST_UTIL.getAdmin().getTableRegions(tableName);
+      // Wait for splits
+      while (tableRegions.size() != 2) {
+        tableRegions = TEST_UTIL.getAdmin().getTableRegions(tableName);
+        Thread.sleep(100);
+      }
+      region.compact(true);
+      Iterator<CachedBlock> iterator = cache.iterator();
+      // Though the split had created the HalfStorefileReader - the firstkey and lastkey scanners
+      // should be closed inorder to return those blocks
+      iterateBlockCache(cache, iterator);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  @Test
   public void testMultiGets() throws IOException, InterruptedException {
     HTable table = null;
     try {


[07/50] [abbrv] hbase git commit: HBASE-15018 Inconsistent way of handling TimeoutException in the rpc client implementations

Posted by jm...@apache.org.
HBASE-15018 Inconsistent way of handling TimeoutException in the rpc client implementations

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


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

Branch: refs/heads/trunk
Commit: 413d663f9262bcdfce67cbc902f7e3153a161fad
Parents: 5e2c2e1
Author: Ashish Singhi <as...@huawei.com>
Authored: Sat Dec 26 22:21:33 2015 +0530
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Dec 29 20:25:47 2015 -0600

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/AbstractRpcClient.java     |  39 ++++++-
 .../apache/hadoop/hbase/ipc/AsyncRpcClient.java |   5 +-
 .../apache/hadoop/hbase/ipc/RpcClientImpl.java  | 102 +++++++------------
 .../hadoop/hbase/ipc/AbstractTestIPC.java       |  18 ++++
 4 files changed, 92 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/413d663f/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 6f5e78a..e33ef3a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -24,6 +24,13 @@ import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.SocketTimeoutException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +41,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.codec.KeyValueCodec;
+import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -41,10 +49,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PoolMap;
 import org.apache.hadoop.io.compress.CompressionCodec;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-
 /**
  * Provides the basics for a RpcClient implementation like configuration and Logging.
  */
@@ -258,6 +262,33 @@ public abstract class AbstractRpcClient implements RpcClient {
   }
 
   /**
+   * Takes an Exception and the address we were trying to connect to and return an IOException with
+   * the input exception as the cause. The new exception provides the stack trace of the place where
+   * the exception is thrown and some extra diagnostics information. If the exception is
+   * ConnectException or SocketTimeoutException, return a new one of the same type; Otherwise return
+   * an IOException.
+   * @param addr target address
+   * @param exception the relevant exception
+   * @return an exception to throw
+   */
+  protected IOException wrapException(InetSocketAddress addr, Exception exception) {
+    if (exception instanceof ConnectException) {
+      // connection refused; include the host:port in the error
+      return (ConnectException) new ConnectException("Call to " + addr
+          + " failed on connection exception: " + exception).initCause(exception);
+    } else if (exception instanceof SocketTimeoutException) {
+      return (SocketTimeoutException) new SocketTimeoutException("Call to " + addr
+          + " failed because " + exception).initCause(exception);
+    } else if (exception instanceof ConnectionClosingException) {
+      return (ConnectionClosingException) new ConnectionClosingException("Call to " + addr
+          + " failed on local exception: " + exception).initCause(exception);
+    } else {
+      return (IOException) new IOException("Call to " + addr + " failed on local exception: "
+          + exception).initCause(exception);
+    }
+  }
+
+  /**
    * Blocking rpc channel that goes via hbase rpc.
    */
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hbase/blob/413d663f/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
index f972d0e..c2bd457 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
@@ -251,10 +251,11 @@ public class AsyncRpcClient extends AbstractRpcClient {
       if (e.getCause() instanceof IOException) {
         throw (IOException) e.getCause();
       } else {
-        throw new IOException(e.getCause());
+        throw wrapException(addr, (Exception) e.getCause());
       }
     } catch (TimeoutException e) {
-      throw new CallTimeoutException(promise.toString());
+      CallTimeoutException cte = new CallTimeoutException(promise.toString());
+      throw wrapException(addr, cte);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/413d663f/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
index d7a0029..ca8a149 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
@@ -19,11 +19,37 @@
 
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Message.Builder;
-import com.google.protobuf.RpcCallback;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.net.SocketFactory;
+import javax.security.sasl.SaslException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -70,37 +96,11 @@ import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
-import javax.net.SocketFactory;
-import javax.security.sasl.SaslException;
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.Closeable;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.net.ConnectException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.net.SocketTimeoutException;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.Message.Builder;
+import com.google.protobuf.RpcCallback;
 
 /**
  * Does RPC against a cluster.  Manages connections per regionserver in the cluster.
@@ -1267,36 +1267,6 @@ public class RpcClientImpl extends AbstractRpcClient {
 
 
   /**
-   * Take an IOException and the address we were trying to connect to
-   * and return an IOException with the input exception as the cause.
-   * The new exception provides the stack trace of the place where
-   * the exception is thrown and some extra diagnostics information.
-   * If the exception is ConnectException or SocketTimeoutException,
-   * return a new one of the same type; Otherwise return an IOException.
-   *
-   * @param addr target address
-   * @param exception the relevant exception
-   * @return an exception to throw
-   */
-  protected IOException wrapException(InetSocketAddress addr,
-                                         IOException exception) {
-    if (exception instanceof ConnectException) {
-      //connection refused; include the host:port in the error
-      return (ConnectException)new ConnectException(
-         "Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
-    } else if (exception instanceof SocketTimeoutException) {
-      return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
-        " failed because " + exception).initCause(exception);
-    } else if (exception instanceof ConnectionClosingException){
-      return (ConnectionClosingException) new ConnectionClosingException(
-          "Call to " + addr + " failed on local exception: " + exception).initCause(exception);
-    } else {
-      return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
-        exception).initCause(exception);
-    }
-  }
-
-  /**
    * Interrupt the connections to the given ip:port server. This should be called if the server
    *  is known as actually dead. This will not prevent current operation to be retried, and,
    *  depending on their own behavior, they may retry on the same server. This can be a feature,

http://git-wip-us.apache.org/repos/asf/hbase/blob/413d663f/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 5df1edc..ffe4d40 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -26,8 +26,10 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
 
 import java.io.IOException;
+import java.net.ConnectException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -42,6 +44,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.MetricsConnection;
+import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
@@ -363,4 +366,19 @@ public abstract class AbstractTestIPC {
       rpcServer.stop();
     }
   }
+
+  @Test
+  public void testWrapException() throws Exception {
+    AbstractRpcClient client =
+        (AbstractRpcClient) RpcClientFactory.createClient(CONF, "AbstractTestIPC");
+    final InetSocketAddress address = InetSocketAddress.createUnresolved("localhost", 0);
+    assertTrue(client.wrapException(address, new ConnectException()) instanceof ConnectException);
+    assertTrue(client.wrapException(address,
+      new SocketTimeoutException()) instanceof SocketTimeoutException);
+    assertTrue(client.wrapException(address, new ConnectionClosingException(
+        "Test AbstractRpcClient#wrapException")) instanceof ConnectionClosingException);
+    assertTrue(client
+        .wrapException(address, new CallTimeoutException("Test AbstractRpcClient#wrapException"))
+        .getCause() instanceof CallTimeoutException);
+  }
 }


[38/50] [abbrv] hbase git commit: HBASE-15065 SimpleRegionNormalizer should return multiple normalization plans in one run

Posted by jm...@apache.org.
HBASE-15065 SimpleRegionNormalizer should return multiple normalization plans in one run


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

Branch: refs/heads/trunk
Commit: 6e781a1f342de069868a8a1f95f6a403a6d4c499
Parents: 164aeb5
Author: tedyu <yu...@gmail.com>
Authored: Fri Jan 8 11:47:34 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 8 11:47:34 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java | 16 ++--
 .../master/normalizer/RegionNormalizer.java     |  4 +-
 .../normalizer/RegionNormalizerChore.java       |  2 +-
 .../normalizer/SimpleRegionNormalizer.java      | 95 +++++++++-----------
 .../normalizer/TestSimpleRegionNormalizer.java  | 27 +++---
 .../TestSimpleRegionNormalizerOnCluster.java    | 16 +++-
 6 files changed, 82 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6e781a1f/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 8ff7ab1..2431681 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
@@ -1341,12 +1341,16 @@ public class HMaster extends HRegionServer implements MasterServices {
             continue;
           }
         }
-        NormalizationPlan plan = this.normalizer.computePlanForTable(table, types);
-        plan.execute(clusterConnection.getAdmin());
-        if (plan.getType() == PlanType.SPLIT) {
-          splitPlanCount++;
-        } else if (plan.getType() == PlanType.MERGE) {
-          mergePlanCount++;
+        List<NormalizationPlan> plans = this.normalizer.computePlanForTable(table, types);
+        if (plans != null) {
+          for (NormalizationPlan plan : plans) {
+            plan.execute(clusterConnection.getAdmin());
+            if (plan.getType() == PlanType.SPLIT) {
+              splitPlanCount++;
+            } else if (plan.getType() == PlanType.MERGE) {
+              mergePlanCount++;
+            }
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e781a1f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
index 616098e..c0083e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizer.java
@@ -51,9 +51,9 @@ public interface RegionNormalizer {
    * Computes next optimal normalization plan.
    * @param table table to normalize
    * @param types desired types of NormalizationPlan
-   * @return Next (perhaps most urgent) normalization action to perform
+   * @return normalization actions to perform. Null if no action to take
    */
-  NormalizationPlan computePlanForTable(TableName table, List<PlanType> types)
+  List<NormalizationPlan> computePlanForTable(TableName table, List<PlanType> types)
       throws HBaseIOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e781a1f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerChore.java
index 25118c7..c6baa2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerChore.java
@@ -38,7 +38,7 @@ public class RegionNormalizerChore extends ScheduledChore {
 
   public RegionNormalizerChore(HMaster master) {
     super(master.getServerName() + "-RegionNormalizerChore", master,
-      master.getConfiguration().getInt("hbase.normalizer.period", 1800000));
+      master.getConfiguration().getInt("hbase.normalizer.period", 300000));
     this.master = master;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e781a1f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index a035647..7195f7d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -72,22 +72,6 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
     this.masterServices = masterServices;
   }
 
-  /*
-   * This comparator compares the region size.
-   * The second element in the triple is region size while the 3rd element
-   * is the index of the region in the underlying List
-   */
-  private Comparator<Triple<HRegionInfo, Long, Integer>> regionSizeComparator =
-      new Comparator<Triple<HRegionInfo, Long, Integer>>() {
-    @Override
-    public int compare(Triple<HRegionInfo, Long, Integer> pair,
-        Triple<HRegionInfo, Long, Integer> pair2) {
-      long sz = pair.getSecond();
-      long sz2 = pair2.getSecond();
-      return (sz < sz2) ? -1 : ((sz == sz2) ? 0 : 1);
-    }
-  };
-
   @Override
   public void planSkipped(HRegionInfo hri, PlanType type) {
     skippedCount[type.ordinal()]++;
@@ -98,6 +82,17 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
     return skippedCount[type.ordinal()];
   }
 
+  // Comparator that gives higher priority to region Split plan
+  private Comparator<NormalizationPlan> planComparator =
+      new Comparator<NormalizationPlan>() {
+    @Override
+    public int compare(NormalizationPlan plan, NormalizationPlan plan2) {
+      if (plan instanceof SplitNormalizationPlan) return -1;
+      if (plan2 instanceof SplitNormalizationPlan) return 1;
+      return 0;
+    }
+  };
+
   /**
    * Computes next most "urgent" normalization action on the table.
    * Action may be either a split, or a merge, or no action.
@@ -107,13 +102,14 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
    * @return normalization plan to execute
    */
   @Override
-  public NormalizationPlan computePlanForTable(TableName table, List<PlanType> types)
+  public List<NormalizationPlan> computePlanForTable(TableName table, List<PlanType> types)
       throws HBaseIOException {
     if (table == null || table.isSystemTable()) {
       LOG.debug("Normalization of system table " + table + " isn't allowed");
-      return EmptyNormalizationPlan.getInstance();
+      return null;
     }
 
+    List<NormalizationPlan> plans = new ArrayList<NormalizationPlan>();
     List<HRegionInfo> tableRegions = masterServices.getAssignmentManager().getRegionStates().
       getRegionsOfTable(table);
 
@@ -122,7 +118,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
       int nrRegions = tableRegions == null ? 0 : tableRegions.size();
       LOG.debug("Table " + table + " has " + nrRegions + " regions, required min number"
         + " of regions for normalizer to run is " + MIN_REGION_COUNT + ", not running normalizer");
-      return EmptyNormalizationPlan.getInstance();
+      return null;
     }
 
     LOG.debug("Computing normalization plan for table: " + table +
@@ -130,56 +126,49 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
 
     long totalSizeMb = 0;
 
-    ArrayList<Triple<HRegionInfo, Long, Integer>> regionsWithSize =
-        new ArrayList<Triple<HRegionInfo, Long, Integer>>(tableRegions.size());
     for (int i = 0; i < tableRegions.size(); i++) {
       HRegionInfo hri = tableRegions.get(i);
       long regionSize = getRegionSize(hri);
-      regionsWithSize.add(new Triple<HRegionInfo, Long, Integer>(hri, regionSize, i));
       totalSizeMb += regionSize;
     }
-    Collections.sort(regionsWithSize, regionSizeComparator);
-
-    Triple<HRegionInfo, Long, Integer> largestRegion = regionsWithSize.get(tableRegions.size()-1);
 
     double avgRegionSize = totalSizeMb / (double) tableRegions.size();
 
     LOG.debug("Table " + table + ", total aggregated regions size: " + totalSizeMb);
     LOG.debug("Table " + table + ", average region size: " + avgRegionSize);
 
-    // now; if the largest region is >2 times large than average, we split it, split
-    // is more high priority normalization action than merge.
-    if (types.contains(PlanType.SPLIT) && largestRegion.getSecond() > 2 * avgRegionSize) {
-      LOG.debug("Table " + table + ", largest region "
-        + largestRegion.getFirst().getRegionNameAsString() + " has size "
-        + largestRegion.getSecond() + ", more than 2 times than avg size, splitting");
-      return new SplitNormalizationPlan(largestRegion.getFirst(), null);
-    }
     int candidateIdx = 0;
-    // look for two successive entries whose indices are adjacent
-    while (candidateIdx < tableRegions.size()-1) {
-      if (Math.abs(regionsWithSize.get(candidateIdx).getThird() -
-        regionsWithSize.get(candidateIdx + 1).getThird()) == 1) {
-        break;
+    while (candidateIdx < tableRegions.size()) {
+      HRegionInfo hri = tableRegions.get(candidateIdx);
+      long regionSize = getRegionSize(hri);
+      // if the region is > 2 times larger than average, we split it, split
+      // is more high priority normalization action than merge.
+      if (types.contains(PlanType.SPLIT) && regionSize > 2 * avgRegionSize) {
+        LOG.debug("Table " + table + ", large region " + hri.getRegionNameAsString() + " has size "
+            + regionSize + ", more than twice avg size, splitting");
+        plans.add(new SplitNormalizationPlan(hri, null));
+      } else {
+        if (candidateIdx == tableRegions.size()-1) {
+          break;
+        }
+        HRegionInfo hri2 = tableRegions.get(candidateIdx+1);
+        long regionSize2 = getRegionSize(hri2);
+        if (types.contains(PlanType.MERGE) && regionSize + regionSize2 < avgRegionSize) {
+          LOG.debug("Table " + table + ", small region size: " + regionSize
+            + " plus its neighbor size: " + regionSize2
+            + ", less than the avg size " + avgRegionSize + ", merging them");
+          plans.add(new MergeNormalizationPlan(hri, hri2));
+          candidateIdx++;
+        }
       }
       candidateIdx++;
     }
-    if (candidateIdx == tableRegions.size()-1) {
-      LOG.debug("No neighboring regions found for table: " + table);
-      return EmptyNormalizationPlan.getInstance();
-    }
-    Triple<HRegionInfo, Long, Integer> candidateRegion = regionsWithSize.get(candidateIdx);
-    Triple<HRegionInfo, Long, Integer> candidateRegion2 = regionsWithSize.get(candidateIdx+1);
-    if (types.contains(PlanType.MERGE) &&
-        candidateRegion.getSecond() + candidateRegion2.getSecond() < avgRegionSize) {
-      LOG.debug("Table " + table + ", smallest region size: " + candidateRegion.getSecond()
-        + " and its smallest neighbor size: " + candidateRegion2.getSecond()
-        + ", less than the avg size, merging them");
-      return new MergeNormalizationPlan(candidateRegion.getFirst(),
-        candidateRegion2.getFirst());
+    if (plans.isEmpty()) {
+      LOG.debug("No normalization needed, regions look good for table: " + table);
+      return null;
     }
-    LOG.debug("No normalization needed, regions look good for table: " + table);
-    return EmptyNormalizationPlan.getInstance();
+    Collections.sort(plans, planComparator);
+    return plans;
   }
 
   private long getRegionSize(HRegionInfo hri) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e781a1f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
index 4395aa3..1f66044 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java
@@ -83,8 +83,8 @@ public class TestSimpleRegionNormalizer {
     Map<byte[], Integer> regionSizes = new HashMap<>();
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
-    assertTrue(plan instanceof EmptyNormalizationPlan);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    assertTrue(plans == null);
   }
 
   @Test
@@ -102,8 +102,8 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri2.getRegionName(), 15);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
-    assertTrue((plan instanceof EmptyNormalizationPlan));
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    assertTrue(plans == null);
   }
 
   @Test
@@ -129,8 +129,8 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri4.getRegionName(), 10);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
-    assertTrue(plan instanceof EmptyNormalizationPlan);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    assertTrue(plans == null);
   }
 
   @Test
@@ -165,15 +165,16 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri5.getRegionName(), 16);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable,
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable,
       mergeDesired ? bothTypes : splitType);
 
     if (mergeDesired) {
+      NormalizationPlan plan = plans.get(0);
       assertTrue(plan instanceof MergeNormalizationPlan);
       assertEquals(hri2, ((MergeNormalizationPlan) plan).getFirstRegion());
       assertEquals(hri3, ((MergeNormalizationPlan) plan).getSecondRegion());
     } else {
-      assertTrue(plan instanceof EmptyNormalizationPlan);
+      assertTrue(plans == null);
     }
   }
 
@@ -209,7 +210,8 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri6.getRegionName(), 2700);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    NormalizationPlan plan = plans.get(0);
 
     assertTrue(plan instanceof MergeNormalizationPlan);
     assertEquals(hri5, ((MergeNormalizationPlan) plan).getFirstRegion());
@@ -243,9 +245,9 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri5.getRegionName(), 5);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
 
-    assertTrue(plan instanceof EmptyNormalizationPlan);
+    assertTrue(plans == null);
   }
 
   @Test
@@ -271,7 +273,8 @@ public class TestSimpleRegionNormalizer {
     regionSizes.put(hri4.getRegionName(), 30);
 
     setupMocksForNormalizer(regionSizes, hris);
-    NormalizationPlan plan = normalizer.computePlanForTable(testTable, bothTypes);
+    List<NormalizationPlan> plans = normalizer.computePlanForTable(testTable, bothTypes);
+    NormalizationPlan plan = plans.get(0);
 
     assertTrue(plan instanceof SplitNormalizationPlan);
     assertEquals(hri4, ((SplitNormalizationPlan) plan).getRegionInfo());

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e781a1f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 99fb268..801e92d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -155,11 +155,19 @@ public class TestSimpleRegionNormalizerOnCluster {
       } while (skippedSplitcnt == 0L);
       assert(skippedSplitcnt > 0);
     } else {
-      while (MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), TABLENAME) < 6) {
-        LOG.info("Waiting for normalization split to complete");
-        Thread.sleep(100);
+      while (true) {
+        List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(TABLENAME);
+        int cnt = 0;
+        for (HRegion region : regions) {
+          String regionName = region.getRegionInfo().getRegionNameAsString();
+          if (regionName.startsWith("testRegionNormalizationSplitOnCluster,zzzzz")) {
+            cnt++;
+          }
+        }
+        if (cnt >= 2) {
+          break;
+        }
       }
-      assertEquals(6, MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), TABLENAME));
     }
 
     admin.disableTable(TABLENAME);


[11/50] [abbrv] hbase git commit: HBASE-15063 Bug in MultiByteBuf#toBytes. (deepankar)

Posted by jm...@apache.org.
HBASE-15063 Bug in MultiByteBuf#toBytes. (deepankar)


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

Branch: refs/heads/trunk
Commit: f01a388a35e03b432b1f3a1ecab04ab2c99e9444
Parents: 92abf8a
Author: anoopsjohn <an...@gmail.com>
Authored: Fri Jan 1 23:11:59 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Fri Jan 1 23:11:59 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/nio/MultiByteBuff.java  | 16 +--------
 .../hadoop/hbase/nio/TestMultiByteBuff.java     | 36 ++++++++++++++++++++
 2 files changed, 37 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f01a388a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
index 06652b8..ab2b5ea 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/MultiByteBuff.java
@@ -1065,21 +1065,7 @@ public class MultiByteBuff extends ByteBuff {
   @Override
   public byte[] toBytes(int offset, int length) {
     byte[] output = new byte[length];
-    int itemIndex = getItemIndex(offset);
-    ByteBuffer item = this.items[itemIndex];
-    int toRead = item.limit() - offset;
-    int destinationOffset = 0;
-    while (length > 0) {
-      toRead = Math.min(length, toRead);
-      ByteBufferUtils.copyFromBufferToArray(output, item, offset, destinationOffset, toRead);
-      length -= toRead;
-      if (length == 0)
-        break;
-      destinationOffset += toRead;
-      offset = 0;
-      item = items[++itemIndex];
-      toRead = item.remaining();
-    }
+    this.get(offset, output, 0, length);
     return output;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f01a388a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
index 193fcff..800c8e1 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/nio/TestMultiByteBuff.java
@@ -342,4 +342,40 @@ public class TestMultiByteBuff {
     assertEquals(2, dst[0]);
     assertEquals(12, mbb1.position());
   }
+
+  @Test
+  public void testToBytes() throws Exception {
+    byte[] b = new byte[4];
+    byte[] b1 = new byte[8];
+    for (int i = 0; i < b.length; i++) {
+      b[i] = (byte) i;
+    }
+    for (int i = 0; i < b1.length; i++) {
+      b1[i] = (byte) (b1.length + i);
+    }
+    ByteBuffer bb1 = ByteBuffer.wrap(b);
+    ByteBuffer bb2 = ByteBuffer.wrap(b1);
+    MultiByteBuff mbb1 = new MultiByteBuff(bb1, bb2);
+
+    // Test 1 Offset hitting exclusive second element
+    byte[] actual = mbb1.toBytes(6, 4);
+    assertTrue(Bytes.equals(actual, 0, actual.length,
+            b1, 2, 4));
+    // Test 2 offset hitting exclusive second element
+    // but continuing to the end of the second one
+    actual = mbb1.toBytes(5, 7);
+    assertTrue(Bytes.equals(actual, 0, actual.length,
+            b1, 1, 7));
+    // Test 3 with offset hitting in first element,
+    // continuing to next
+    actual = mbb1.toBytes(2, 7);
+    byte[] expected = new byte[7];
+    System.arraycopy(b, 2, expected, 0,  2);
+    System.arraycopy(b1, 0, expected, 2, 5);
+    assertTrue(Bytes.equals(actual, expected));
+    // Test 4 hitting only in first exclusively
+    actual = mbb1.toBytes(1, 3);
+    assertTrue(Bytes.equals(actual, 0, actual.length,
+            b, 1, 3));
+  }
 }


[44/50] [abbrv] hbase git commit: HBASE-15077 Support OffheapKV write in compaction with out copying data on heap.

Posted by jm...@apache.org.
HBASE-15077 Support OffheapKV write in compaction with out copying data on heap.


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

Branch: refs/heads/trunk
Commit: da932ee38d29a13acb3e3ff6653402d4c8141d04
Parents: ec47a81
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Jan 12 10:02:39 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Tue Jan 12 10:02:39 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/OffheapKeyValue.java    |   2 +-
 .../hadoop/hbase/io/ByteArrayOutputStream.java  | 129 +++++++++++++++++++
 .../hadoop/hbase/io/ByteBufferOutputStream.java |   3 +-
 .../io/ByteBufferSupportDataOutputStream.java   |  44 +++++++
 .../hbase/io/ByteBufferSupportOutputStream.java |  51 ++++++++
 .../io/encoding/BufferedDataBlockEncoder.java   |   4 +-
 .../hadoop/hbase/util/ByteBufferUtils.java      |  26 +---
 .../hadoop/hbase/io/hfile/HFileBlock.java       |  23 +---
 .../io/encoding/TestDataBlockEncoders.java      |  11 +-
 .../io/hfile/TestHFileDataBlockEncoder.java     |   7 +-
 10 files changed, 245 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
index 0af64cd..ced1595 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
@@ -254,7 +254,7 @@ public class OffheapKeyValue extends ByteBufferedCell implements HeapSize, Clone
       length = keyLen + this.getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
     }
     ByteBufferUtils.putInt(out, length);
-    ByteBufferUtils.writeByteBuffer(out, this.buf, this.offset, length);
+    ByteBufferUtils.copyBufferToStream(out, this.buf, this.offset, length);
     return length + Bytes.SIZEOF_INT;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteArrayOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteArrayOutputStream.java
new file mode 100644
index 0000000..d951595
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteArrayOutputStream.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Our own implementation of ByteArrayOutputStream where all methods are NOT synchronized and
+ * supports writing ByteBuffer directly to it.
+ */
+@InterfaceAudience.Private
+public class ByteArrayOutputStream extends OutputStream implements ByteBufferSupportOutputStream {
+
+  // Borrowed from openJDK:
+  // 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;
+
+  private byte[] buf;
+  private int pos = 0;
+
+  public ByteArrayOutputStream() {
+    this(32);
+  }
+
+  public ByteArrayOutputStream(int capacity) {
+    this.buf = new byte[capacity];
+  }
+
+  @Override
+  public void write(ByteBuffer b, int off, int len) throws IOException {
+    checkSizeAndGrow(len);
+    ByteBufferUtils.copyFromBufferToArray(this.buf, b, off, this.pos, len);
+    this.pos += len;
+  }
+
+  @Override
+  public void writeInt(int i) throws IOException {
+    checkSizeAndGrow(Bytes.SIZEOF_INT);
+    Bytes.putInt(this.buf, this.pos, i);
+    this.pos += Bytes.SIZEOF_INT;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    checkSizeAndGrow(Bytes.SIZEOF_BYTE);
+    buf[this.pos] = (byte) b;
+    this.pos++;
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    checkSizeAndGrow(len);
+    System.arraycopy(b, off, this.buf, this.pos, len);
+    this.pos += len;
+  }
+
+  private void checkSizeAndGrow(int extra) {
+    long capacityNeeded = this.pos + (long) extra;
+    if (capacityNeeded > this.buf.length) {
+      // guarantee it's possible to fit
+      if (capacityNeeded > MAX_ARRAY_SIZE) {
+        throw new BufferOverflowException();
+      }
+      // double until hit the cap
+      long nextCapacity = Math.min(this.buf.length << 1, MAX_ARRAY_SIZE);
+      // but make sure there is enough if twice the existing capacity is still too small
+      nextCapacity = Math.max(nextCapacity, capacityNeeded);
+      if (nextCapacity > MAX_ARRAY_SIZE) {
+        throw new BufferOverflowException();
+      }
+      byte[] newBuf = new byte[(int) nextCapacity];
+      System.arraycopy(buf, 0, newBuf, 0, buf.length);
+      buf = newBuf;
+    }
+  }
+
+  /**
+   * Resets the <code>pos</code> field of this byte array output stream to zero. The output stream
+   * can be used again.
+   */
+  public void reset() {
+    this.pos = 0;
+  }
+
+  /**
+   * Copies the content of this Stream into a new byte array.
+   * @return  the contents of this output stream, as new byte array.
+   */
+  public byte toByteArray()[] {
+    return Arrays.copyOf(buf, pos);
+  }
+
+  /**
+   * @return the underlying array where the data gets accumulated
+   */
+  public byte[] getBuffer() {
+    return this.buf;
+  }
+
+  /**
+   * @return The current size of the buffer.
+   */
+  public int size() {
+    return this.pos;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/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 d91513e..d4bda18 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
@@ -37,7 +37,8 @@ import org.apache.hadoop.hbase.util.Bytes;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class ByteBufferOutputStream extends OutputStream {
+public class ByteBufferOutputStream extends OutputStream
+    implements ByteBufferSupportOutputStream {
   
   // Borrowed from openJDK:
   // http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/8-b132/java/util/ArrayList.java#221

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportDataOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportDataOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportDataOutputStream.java
new file mode 100644
index 0000000..3a52e63
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportDataOutputStream.java
@@ -0,0 +1,44 @@
+/**
+ * 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.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * Our extension of DataOutputStream which implements ByteBufferSupportOutputStream
+ */
+@InterfaceAudience.Private
+public class ByteBufferSupportDataOutputStream extends DataOutputStream
+    implements ByteBufferSupportOutputStream {
+
+  public ByteBufferSupportDataOutputStream(OutputStream out) {
+    super(out);
+  }
+
+  @Override
+  public void write(ByteBuffer b, int off, int len) throws IOException {
+    ByteBufferUtils.copyBufferToStream(out, b, off, len);
+    written += len;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportOutputStream.java
new file mode 100644
index 0000000..ccb5c81
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportOutputStream.java
@@ -0,0 +1,51 @@
+/**
+ * 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.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Interface adds support for writing {@link ByteBuffer} into OutputStream.
+ */
+@InterfaceAudience.Private
+public interface ByteBufferSupportOutputStream {
+
+  /**
+   * Writes <code>len</code> bytes from the specified ByteBuffer starting at offset <code>off</code>
+   * to this output stream.
+   *
+   * @param b the data.
+   * @param off the start offset in the data.
+   * @param len the number of bytes to write.
+   * @exception IOException
+   *              if an I/O error occurs. In particular, an <code>IOException</code> is thrown if
+   *              the output stream is closed.
+   */
+  void write(ByteBuffer b, int off, int len) throws IOException;
+
+  /**
+   * Writes an <code>int</code> to the underlying output stream as four
+   * bytes, high byte first.
+   * @param i the <code>int</code> to write
+   * @throws IOException if an I/O error occurs.
+   */
+  void writeInt(int i) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index 112f258..33e38c7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -673,14 +673,14 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
       // Write key
       out.write(keyBuffer.array());
       // Write value
-      ByteBufferUtils.writeByteBuffer(out, this.valueBuffer, this.valueOffset, this.valueLength);
+      ByteBufferUtils.copyBufferToStream(out, this.valueBuffer, this.valueOffset, this.valueLength);
       if (withTags) {
         // 2 bytes tags length followed by tags bytes
         // tags length is serialized with 2 bytes only(short way) even if the type is int.
         // As this is non -ve numbers, we save the sign bit. See HBASE-11437
         out.write((byte) (0xff & (this.tagsLength >> 8)));
         out.write((byte) (0xff & this.tagsLength));
-        ByteBufferUtils.writeByteBuffer(out, this.tagsBuffer, this.tagsOffset, this.tagsLength);
+        ByteBufferUtils.copyBufferToStream(out, this.tagsBuffer, this.tagsOffset, this.tagsLength);
       }
       return lenToWrite + Bytes.SIZEOF_INT;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 6e3fcaa..62173c2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -27,7 +27,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
+import org.apache.hadoop.hbase.io.ByteBufferSupportOutputStream;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -141,8 +141,8 @@ public final class ByteBufferUtils {
      // We have writeInt in ByteBufferOutputStream so that it can directly write
      // int to underlying
      // ByteBuffer in one step.
-     if (out instanceof ByteBufferOutputStream) {
-       ((ByteBufferOutputStream) out).writeInt(value);
+     if (out instanceof ByteBufferSupportOutputStream) {
+       ((ByteBufferSupportOutputStream) out).writeInt(value);
      } else {
        StreamUtils.writeInt(out, value);
      }
@@ -179,9 +179,10 @@ public final class ByteBufferUtils {
    */
   public static void copyBufferToStream(OutputStream out, ByteBuffer in,
       int offset, int length) throws IOException {
-    if (in.hasArray()) {
-      out.write(in.array(), in.arrayOffset() + offset,
-          length);
+    if (out instanceof ByteBufferSupportOutputStream) {
+      ((ByteBufferSupportOutputStream) out).write(in, offset, length);
+    } else if (in.hasArray()) {
+      out.write(in.array(), in.arrayOffset() + offset, length);
     } else {
       for (int i = 0; i < length; ++i) {
         out.write(toByte(in, offset + i));
@@ -904,19 +905,6 @@ public final class ByteBufferUtils {
     }
   }
 
-  public static void writeByteBuffer(OutputStream out, ByteBuffer b, int offset, int length)
-      throws IOException {
-    // We have write which takes ByteBuffer in ByteBufferOutputStream so that it
-    // can directly write
-    // bytes from the src ByteBuffer to the destination ByteBuffer. This avoid
-    // need for temp array
-    // creation and copy
-    if (out instanceof ByteBufferOutputStream) {
-      ((ByteBufferOutputStream) out).write(b, offset, length);
-    } else {
-      ByteBufferUtils.copyBufferToStream(out, b, offset, length);
-    }
-  }
   // For testing purpose
   public static String toStringBinary(final ByteBuffer b, int off, int len) {
     StringBuilder result = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index a68d0a6..6916ba0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
@@ -35,7 +34,9 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
 import org.apache.hadoop.hbase.io.ByteBuffInputStream;
+import org.apache.hadoop.hbase.io.ByteBufferSupportDataOutputStream;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
@@ -915,7 +916,7 @@ public class HFileBlock implements Cacheable {
       state = State.WRITING;
 
       // We will compress it later in finishBlock()
-      userDataStream = new DataOutputStream(baosInMemory);
+      userDataStream = new ByteBufferSupportDataOutputStream(baosInMemory);
       if (newBlockType == BlockType.DATA) {
         this.dataBlockEncoder.startBlockEncoding(dataBlockEncodingCtx, userDataStream);
       }
@@ -969,11 +970,8 @@ public class HFileBlock implements Cacheable {
      */
     private void finishBlock() throws IOException {
       if (blockType == BlockType.DATA) {
-        BufferGrabbingByteArrayOutputStream baosInMemoryCopy =
-            new BufferGrabbingByteArrayOutputStream();
-        baosInMemory.writeTo(baosInMemoryCopy);
         this.dataBlockEncoder.endBlockEncoding(dataBlockEncodingCtx, userDataStream,
-            baosInMemoryCopy.buf, blockType);
+            baosInMemory.getBuffer(), blockType);
         blockType = dataBlockEncodingCtx.getBlockType();
       }
       userDataStream.flush();
@@ -1011,19 +1009,6 @@ public class HFileBlock implements Cacheable {
           onDiskChecksum, 0, fileContext.getChecksumType(), fileContext.getBytesPerChecksum());
     }
 
-    public static class BufferGrabbingByteArrayOutputStream extends ByteArrayOutputStream {
-      private byte[] buf;
-
-      @Override
-      public void write(byte[] b, int off, int len) {
-        this.buf = b;
-      }
-
-      public byte[] getBuffer() {
-        return this.buf;
-      }
-    }
-
     /**
      * Put the header into the given byte array at the given offset.
      * @param onDiskSize size of the block on disk header + data + checksum

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
index 00969b2..1ef918c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -45,8 +44,8 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeSeeker;
+import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
 import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.hfile.HFileBlock.Writer.BufferGrabbingByteArrayOutputStream;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
@@ -256,9 +255,7 @@ public class TestDataBlockEncoders {
     for (KeyValue kv : kvs) {
       encoder.encode(kv, encodingContext, dos);
     }
-    BufferGrabbingByteArrayOutputStream stream = new BufferGrabbingByteArrayOutputStream();
-    baos.writeTo(stream);
-    encoder.endBlockEncoding(encodingContext, dos, stream.getBuffer());
+    encoder.endBlockEncoding(encodingContext, dos, baos.getBuffer());
     byte[] encodedData = new byte[baos.size() - ENCODED_DATA_OFFSET];
     System.arraycopy(baos.toByteArray(), ENCODED_DATA_OFFSET, encodedData, 0, encodedData.length);
     if (useOffheapData) {
@@ -398,9 +395,7 @@ public class TestDataBlockEncoders {
       for (KeyValue kv : kvList) {
         encoder.encode(kv, encodingContext, dos);
       }
-      BufferGrabbingByteArrayOutputStream stream = new BufferGrabbingByteArrayOutputStream();
-      baos.writeTo(stream);
-      encoder.endBlockEncoding(encodingContext, dos, stream.getBuffer());
+      encoder.endBlockEncoding(encodingContext, dos, baos.getBuffer());
       byte[] encodedData = baos.toByteArray();
 
       testAlgorithm(encodedData, unencodedDataBuf, encoder);

http://git-wip-us.apache.org/repos/asf/hbase/blob/da932ee3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
index 2523a8c..6f434bb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
@@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -31,12 +30,12 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
-import org.apache.hadoop.hbase.io.hfile.HFileBlock.Writer.BufferGrabbingByteArrayOutputStream;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.ChecksumType;
@@ -217,9 +216,7 @@ public class TestHFileDataBlockEncoder {
     for (KeyValue kv : kvs) {
       blockEncoder.encode(kv, context, dos);
     }
-    BufferGrabbingByteArrayOutputStream stream = new BufferGrabbingByteArrayOutputStream();
-    baos.writeTo(stream);
-    blockEncoder.endBlockEncoding(context, dos, stream.getBuffer(), BlockType.DATA);
+    blockEncoder.endBlockEncoding(context, dos, baos.getBuffer(), BlockType.DATA);
     byte[] encodedBytes = baos.toByteArray();
     size = encodedBytes.length - block.getDummyHeaderForVersion().length;
     return new HFileBlock(context.getBlockType(), size, size, -1, ByteBuffer.wrap(encodedBytes),


[42/50] [abbrv] hbase git commit: HBASE-15027 - Findbug fix (addendum)

Posted by jm...@apache.org.
HBASE-15027 - Findbug fix (addendum)


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

Branch: refs/heads/trunk
Commit: 894ce84b8330f9e3b1ca5b04e591c7a77cd2ce13
Parents: c2f67a0
Author: ramkrishna <ra...@gmail.com>
Authored: Mon Jan 11 16:50:59 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Mon Jan 11 16:50:59 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/CompactedHFilesDischarger.java      | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/894ce84b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
index c4974cf..18cf35d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
@@ -73,6 +73,9 @@ public class CompactedHFilesDischarger extends ScheduledChore {
 
   @Override
   public void chore() {
+    // Noop if rss is null. This will never happen in a normal condition except for cases
+    // when the test case is not spinning up a cluster
+    if (regionServerServices == null) return;
     List<Region> onlineRegions = regionServerServices.getOnlineRegions();
     if (onlineRegions != null) {
       for (Region region : onlineRegions) {


[25/50] [abbrv] hbase git commit: HBASE-15064 BufferUnderflowException after last Cell fetched from an HFile Block served from L2 offheap cache.

Posted by jm...@apache.org.
HBASE-15064 BufferUnderflowException after last Cell fetched from an HFile Block served from L2 offheap cache.


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

Branch: refs/heads/trunk
Commit: 7cd09bfb91e829a7595b97842e64cfd8edcd97f9
Parents: e8fbc9b
Author: anoopsjohn <an...@gmail.com>
Authored: Wed Jan 6 10:25:32 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Wed Jan 6 10:25:32 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/util/ByteBufferArray.java      |  5 +++
 .../hadoop/hbase/util/TestByteBufferArray.java  | 43 ++++++++++++++++++++
 2 files changed, 48 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7cd09bfb/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
index 986d6e0..2334cf7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
@@ -219,6 +219,11 @@ public final class ByteBufferArray {
     long end = offset + len;
     int startBuffer = (int) (offset / bufferSize), startBufferOffset = (int) (offset % bufferSize);
     int endBuffer = (int) (end / bufferSize), endBufferOffset = (int) (end % bufferSize);
+    // Last buffer in the array is a dummy one with 0 capacity. Avoid sending back that
+    if (endBuffer == this.bufferCount) {
+      endBuffer--;
+      endBufferOffset = bufferSize;
+    }
     assert startBuffer >= 0 && startBuffer < bufferCount;
     assert endBuffer >= 0 && endBuffer < bufferCount
         || (endBuffer == bufferCount && endBufferOffset == 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7cd09bfb/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
new file mode 100644
index 0000000..701601d
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestByteBufferArray {
+
+  @Test
+  public void testAsSubBufferWhenEndOffsetLandInLastBuffer() throws Exception {
+    int capacity = 4 * 1024 * 1024;
+    ByteBufferArray array = new ByteBufferArray(capacity, false);
+    ByteBuff subBuf = array.asSubByteBuff(0, capacity);
+    subBuf.position(capacity - 1);// Position to the last byte
+    assertTrue(subBuf.hasRemaining());
+    // Read last byte
+    subBuf.get();
+    assertFalse(subBuf.hasRemaining());
+  }
+}


[02/50] [abbrv] hbase git commit: HBASE-14796 Enhance the Gets in the connector (Zhan Zhang)

Posted by jm...@apache.org.
HBASE-14796 Enhance the Gets in the connector (Zhan Zhang)


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

Branch: refs/heads/trunk
Commit: 6868c6366002d5b4e25980f37ede8839e7a7e92d
Parents: 2fba25b
Author: tedyu <yu...@gmail.com>
Authored: Mon Dec 28 15:48:10 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Dec 28 15:48:10 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/spark/DefaultSource.scala      | 33 ++-----
 .../hadoop/hbase/spark/datasources/Bound.scala  | 24 +++++
 .../spark/datasources/HBaseResources.scala      | 14 +++
 .../spark/datasources/HBaseSparkConf.scala      |  2 +
 .../spark/datasources/HBaseTableScanRDD.scala   | 92 +++++++++++++++++---
 5 files changed, 127 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6868c636/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
index 73cab3c..b6d7982 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
@@ -159,6 +159,10 @@ case class HBaseRelation (val tableName:String,
     .getOrElse(sqlContext.sparkContext.getConf.getInt(
     HBaseSparkConf.BATCH_NUM,  HBaseSparkConf.defaultBatchNum))
 
+  val bulkGetSize =  parameters.get(HBaseSparkConf.BULKGET_SIZE).map(_.toInt)
+    .getOrElse(sqlContext.sparkContext.getConf.getInt(
+    HBaseSparkConf.BULKGET_SIZE,  HBaseSparkConf.defaultBulkGetSize))
+
   //create or get latest HBaseContext
   @transient val hbaseContext:HBaseContext = if (useHBaseContext) {
     LatestHBaseContextCache.latest
@@ -267,6 +271,7 @@ case class HBaseRelation (val tableName:String,
       None
     }
     val hRdd = new HBaseTableScanRDD(this, pushDownFilterJava, requiredQualifierDefinitionList.seq)
+    pushDownRowKeyFilter.points.foreach(hRdd.addPoint(_))
     pushDownRowKeyFilter.ranges.foreach(hRdd.addRange(_))
     var resultRDD: RDD[Row] = {
       val tmp = hRdd.map{ r =>
@@ -280,34 +285,6 @@ case class HBaseRelation (val tableName:String,
       }
     }
 
-    //If there are gets then we can get them from the driver and union that rdd in
-    // with the rest of the values.
-    if (getList.size() > 0) {
-      val connection =
-        ConnectionFactory.createConnection(hbaseContext.tmpHdfsConfiguration)
-      try {
-        val table = connection.getTable(TableName.valueOf(tableName))
-        try {
-          val results = table.get(getList)
-          val rowList = mutable.MutableList[Row]()
-          for (i <- 0 until results.length) {
-            val rowArray = requiredColumns.map(c =>
-              DefaultSourceStaticUtils.getValue(c, schemaMappingDefinition, results(i)))
-            rowList += Row.fromSeq(rowArray)
-          }
-          val getRDD = sqlContext.sparkContext.parallelize(rowList)
-          if (resultRDD == null) resultRDD = getRDD
-          else {
-            resultRDD = resultRDD.union(getRDD)
-          }
-        } finally {
-          table.close()
-        }
-      } finally {
-        connection.close()
-      }
-    }
-
     if (resultRDD == null) {
       val scan = new Scan()
       scan.setCacheBlocks(blockCacheEnable)

http://git-wip-us.apache.org/repos/asf/hbase/blob/6868c636/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
index 0f6098d..8e03e95 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
@@ -87,3 +87,27 @@ object Ranges {
   }
 }
 
+object Points {
+  def and(r: Range, ps: Seq[Array[Byte]]): Seq[Array[Byte]] = {
+    ps.flatMap { p =>
+      if (ord.compare(r.lower.get.b, p) <= 0) {
+        // if region lower bound is less or equal to the point
+        if (r.upper.isDefined) {
+          // if region upper bound is defined
+          if (ord.compare(r.upper.get.b, p) > 0) {
+            // if the upper bound is greater than the point (because upper bound is exclusive)
+            Some(p)
+          } else {
+            None
+          }
+        } else {
+          // if the region upper bound is not defined (infinity)
+          Some(p)
+        }
+      } else {
+        None
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/6868c636/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
index 19a6ea7..14c5fd0 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
@@ -38,6 +38,12 @@ case class ScanResource(tbr: TableResource, rs: ResultScanner) extends Resource
   }
 }
 
+case class GetResource(tbr: TableResource, rs: Array[Result]) extends Resource {
+  def release() {
+    tbr.release()
+  }
+}
+
 trait ReferencedResource {
   var count: Int = 0
   def init(): Unit
@@ -100,6 +106,10 @@ case class TableResource(relation: HBaseRelation) extends ReferencedResource {
   def getScanner(scan: Scan): ScanResource = releaseOnException {
     ScanResource(this, table.getScanner(scan))
   }
+
+  def get(list: java.util.List[org.apache.hadoop.hbase.client.Get]) = releaseOnException {
+    GetResource(this, table.get(list))
+  }
 }
 
 case class RegionResource(relation: HBaseRelation) extends ReferencedResource {
@@ -138,6 +148,10 @@ object HBaseResources{
     sr.rs
   }
 
+  implicit def GetResToResult(gr: GetResource): Array[Result] = {
+    gr.rs
+  }
+
   implicit def TableResToTable(tr: TableResource): Table = {
     tr.table
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6868c636/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
index 67580b0..5e11356 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
@@ -29,4 +29,6 @@ object HBaseSparkConf{
   val defaultCachingSize = 1000
   val BATCH_NUM = "spark.hbase.batchNum"
   val defaultBatchNum = 1000
+  val BULKGET_SIZE = "spark.hbase.bulkGetSize"
+  val defaultBulkGetSize = 1000
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6868c636/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
index eb9d39a..f288c34 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
@@ -17,6 +17,8 @@
 
 package org.apache.hadoop.hbase.spark.datasources
 
+import java.util.ArrayList
+
 import org.apache.hadoop.hbase.client._
 import org.apache.hadoop.hbase.spark.{ScanRange, SchemaQualifierDefinition, HBaseRelation, SparkSQLPushDownFilter}
 import org.apache.hadoop.hbase.spark.hbase._
@@ -32,7 +34,12 @@ class HBaseTableScanRDD(relation: HBaseRelation,
      val columns: Seq[SchemaQualifierDefinition] = Seq.empty
      )extends RDD[Result](relation.sqlContext.sparkContext, Nil) with Logging  {
   private def sparkConf = SparkEnv.get.conf
-  var ranges = Seq.empty[Range]
+  @transient var ranges = Seq.empty[Range]
+  @transient var points = Seq.empty[Array[Byte]]
+  def addPoint(p: Array[Byte]) {
+    points :+= p
+  }
+
   def addRange(r: ScanRange) = {
     val lower = if (r.lowerBound != null && r.lowerBound.length > 0) {
       Some(Bound(r.lowerBound, r.isLowerBoundEqualTo))
@@ -65,12 +72,13 @@ class HBaseTableScanRDD(relation: HBaseRelation,
     logDebug(s"There are ${regions.size} regions")
     val ps = regions.flatMap { x =>
       val rs = Ranges.and(Range(x), ranges)
-      if (rs.size > 0) {
+      val ps = Points.and(Range(x), points)
+      if (rs.size > 0 || ps.size > 0) {
         if(log.isDebugEnabled) {
           rs.foreach(x => logDebug(x.toString))
         }
         idx += 1
-        Some(HBaseScanPartition(idx - 1, x, rs, SerializedFilter.toSerializedTypedFilter(filter)))
+        Some(HBaseScanPartition(idx - 1, x, rs, ps, SerializedFilter.toSerializedTypedFilter(filter)))
       } else {
         None
       }
@@ -86,6 +94,57 @@ class HBaseTableScanRDD(relation: HBaseRelation,
     }.toSeq
   }
 
+  private def buildGets(
+      tbr: TableResource,
+      g: Seq[Array[Byte]],
+      filter: Option[SparkSQLPushDownFilter],
+      columns: Seq[SchemaQualifierDefinition]): Iterator[Result] = {
+    g.grouped(relation.bulkGetSize).flatMap{ x =>
+      val gets = new ArrayList[Get]()
+      x.foreach{ y =>
+        val g = new Get(y)
+        columns.foreach { d =>
+          if (d.columnFamilyBytes.length > 0) {
+            g.addColumn(d.columnFamilyBytes, d.qualifierBytes)
+          }
+        }
+        filter.foreach(g.setFilter(_))
+        gets.add(g)
+      }
+      val tmp = tbr.get(gets)
+      rddResources.addResource(tmp)
+      toResultIterator(tmp)
+    }
+  }
+
+  private def toResultIterator(result: GetResource): Iterator[Result] = {
+    val iterator = new Iterator[Result] {
+      var idx = 0
+      var cur: Option[Result] = None
+      override def hasNext: Boolean = {
+        while(idx < result.length && cur.isEmpty) {
+          val r = result(idx)
+          idx += 1
+          if (!r.isEmpty) {
+            cur = Some(r)
+          }
+        }
+        if (cur.isEmpty) {
+          rddResources.release(result)
+        }
+        cur.isDefined
+      }
+      override def next(): Result = {
+        hasNext
+        val ret = cur.get
+        cur = None
+        ret
+      }
+    }
+    iterator
+  }
+
+
   private def buildScan(range: Range,
       filter: Option[SparkSQLPushDownFilter],
       columns: Seq[SchemaQualifierDefinition]): Scan = {
@@ -130,6 +189,7 @@ class HBaseTableScanRDD(relation: HBaseRelation,
     }
     iterator
   }
+
   lazy val rddResources = RDDResources(new mutable.HashSet[Resource]())
 
   private def close() {
@@ -138,18 +198,29 @@ class HBaseTableScanRDD(relation: HBaseRelation,
 
   override def compute(split: Partition, context: TaskContext): Iterator[Result] = {
     val partition = split.asInstanceOf[HBaseScanPartition]
-
+    val filter = SerializedFilter.fromSerializedFilter(partition.sf)
     val scans = partition.scanRanges
-      .map(buildScan(_, SerializedFilter.fromSerializedFilter(partition.sf), columns))
+      .map(buildScan(_, filter, columns))
     val tableResource = TableResource(relation)
     context.addTaskCompletionListener(context => close())
-    val sIts = scans.par
-      .map(tableResource.getScanner(_))
-      .map(toResultIterator(_))
+    val points = partition.points
+    val gIt: Iterator[Result] =  {
+      if (points.isEmpty) {
+        Iterator.empty: Iterator[Result]
+      } else {
+        buildGets(tableResource, points, filter, columns)
+      }
+    }
+    val rIts = scans.par
+      .map { scan =>
+      val scanner = tableResource.getScanner(scan)
+      rddResources.addResource(scanner)
+      scanner
+    }.map(toResultIterator(_))
       .fold(Iterator.empty: Iterator[Result]){ case (x, y) =>
       x ++ y
-    }
-    sIts
+    } ++ gIt
+    rIts
   }
 }
 
@@ -176,6 +247,7 @@ private[hbase] case class HBaseScanPartition(
     override val index: Int,
     val regions: HBaseRegion,
     val scanRanges: Seq[Range],
+    val points: Seq[Array[Byte]],
     val sf: SerializedFilter) extends Partition
 
 case class RDDResources(set: mutable.HashSet[Resource]) {


[15/50] [abbrv] hbase git commit: HBASE-14987 Compaction marker whose region name doesn't match current region's needs to be handled

Posted by jm...@apache.org.
HBASE-14987 Compaction marker whose region name doesn't match current region's needs to be handled


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

Branch: refs/heads/trunk
Commit: 00656688f73c85ea9e6f5241ac852f72e774eeea
Parents: 9589a7d
Author: tedyu <yu...@gmail.com>
Authored: Mon Jan 4 07:10:10 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jan 4 07:10:10 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  9 +++-
 .../hadoop/hbase/regionserver/HRegion.java      | 44 ++++++++++++++++----
 .../hadoop/hbase/regionserver/TestHRegion.java  | 26 ++++++++++--
 3 files changed, 65 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/00656688/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index b2d5994..c02309b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -2563,12 +2563,19 @@ public final class ProtobufUtil {
   @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
+    return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir);
+  }
+
+  @SuppressWarnings("deprecation")
+  public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] regionName,
+      byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     // compaction descriptor contains relative paths.
     // input / output paths are relative to the store dir
     // store dir is relative to region dir
     CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
         .setTableName(ByteStringer.wrap(info.getTable().toBytes()))
-        .setEncodedRegionName(ByteStringer.wrap(info.getEncodedNameAsBytes()))
+        .setEncodedRegionName(ByteStringer.wrap(
+          regionName == null ? info.getEncodedNameAsBytes() : regionName))
         .setFamilyName(ByteStringer.wrap(family))
         .setStoreHomeDir(storeDir.getName()); //make relative
     for (Path inputPath : inputPaths) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/00656688/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 9549a13..ccf2eb0 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
@@ -4130,11 +4130,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               continue;
             }
           }
+          boolean checkRowWithinBoundary = false;
           // Check this edit is for this region.
           if (!Bytes.equals(key.getEncodedRegionName(),
               this.getRegionInfo().getEncodedNameAsBytes())) {
-            skippedEdits++;
-            continue;
+            checkRowWithinBoundary = true;
           }
 
           boolean flush = false;
@@ -4142,11 +4142,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             // Check this edit is for me. Also, guard against writing the special
             // METACOLUMN info such as HBASE::CACHEFLUSH entries
             if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
-              //this is a special edit, we should handle it
-              CompactionDescriptor compaction = WALEdit.getCompaction(cell);
-              if (compaction != null) {
-                //replay the compaction
-                replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
+              // if region names don't match, skipp replaying compaction marker
+              if (!checkRowWithinBoundary) {
+                //this is a special edit, we should handle it
+                CompactionDescriptor compaction = WALEdit.getCompaction(cell);
+                if (compaction != null) {
+                  //replay the compaction
+                  replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
+                }
               }
               skippedEdits++;
               continue;
@@ -4162,6 +4165,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               skippedEdits++;
               continue;
             }
+            if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
+              cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
+              LOG.warn("Row of " + cell + " is not within region boundary");
+              skippedEdits++;
+              continue;
+            }
             // Now, figure if we should skip this edit.
             if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
                 .getName())) {
@@ -4232,8 +4241,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
       boolean removeFiles, long replaySeqId)
       throws IOException {
-    checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
-      "Compaction marker from WAL ", compaction);
+    try {
+      checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
+        "Compaction marker from WAL ", compaction);
+    } catch (WrongRegionException wre) {
+      if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
+        // skip the compaction marker since it is not for this region
+        return;
+      }
+      throw wre;
+    }
 
     synchronized (writestate) {
       if (replaySeqId < lastReplayedOpenRegionSeqId) {
@@ -6590,6 +6607,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             (Bytes.compareTo(info.getEndKey(), row) > 0));
   }
 
+  public static boolean rowIsInRange(HRegionInfo info, final byte [] row, final int offset,
+      final short length) {
+    return ((info.getStartKey().length == 0) ||
+        (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
+          row, offset, length) <= 0)) &&
+        ((info.getEndKey().length == 0) ||
+          (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
+  }
+
   /**
    * Merge two HRegions.  The regions must be adjacent and must not overlap.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/00656688/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 35de488..4582e31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -839,6 +839,10 @@ public class TestHRegion {
 
   @Test
   public void testRecoveredEditsReplayCompaction() throws Exception {
+    testRecoveredEditsReplayCompaction(false);
+    testRecoveredEditsReplayCompaction(true);
+  }
+  public void testRecoveredEditsReplayCompaction(boolean mismatchedRegionName) throws Exception {
     String method = name.getMethodName();
     TableName tableName = TableName.valueOf(method);
     byte[] family = Bytes.toBytes("family");
@@ -884,9 +888,17 @@ public class TestHRegion {
       Path newFile = region.getRegionFileSystem().commitStoreFile(Bytes.toString(family),
           files[0].getPath());
 
+      byte[] encodedNameAsBytes = this.region.getRegionInfo().getEncodedNameAsBytes();
+      byte[] fakeEncodedNameAsBytes = new byte [encodedNameAsBytes.length];
+      for (int i=0; i < encodedNameAsBytes.length; i++) {
+        // Mix the byte array to have a new encodedName
+        fakeEncodedNameAsBytes[i] = (byte) (encodedNameAsBytes[i] + 1);
+      }
+
       CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(this.region
-          .getRegionInfo(), family, storeFiles, Lists.newArrayList(newFile), region
-          .getRegionFileSystem().getStoreDir(Bytes.toString(family)));
+        .getRegionInfo(), mismatchedRegionName ? fakeEncodedNameAsBytes : null, family,
+            storeFiles, Lists.newArrayList(newFile),
+            region.getRegionFileSystem().getStoreDir(Bytes.toString(family)));
 
       WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(),
           this.region.getRegionInfo(), compactionDescriptor, region.getMVCC());
@@ -908,14 +920,20 @@ public class TestHRegion {
       region.getTableDesc();
       region.getRegionInfo();
       region.close();
-      region = HRegion.openHRegion(region, null);
+      try {
+        region = HRegion.openHRegion(region, null);
+      } catch (WrongRegionException wre) {
+        fail("Matching encoded region name should not have produced WrongRegionException");
+      }
 
       // now check whether we have only one store file, the compacted one
       Collection<StoreFile> sfs = region.getStore(family).getStorefiles();
       for (StoreFile sf : sfs) {
         LOG.info(sf.getPath());
       }
-      assertEquals(1, region.getStore(family).getStorefilesCount());
+      if (!mismatchedRegionName) {
+        assertEquals(1, region.getStore(family).getStorefilesCount());
+      }
       files = FSUtils.listStatus(fs, tmpDir);
       assertTrue("Expected to find 0 files inside " + tmpDir, files == null || files.length == 0);
 


[23/50] [abbrv] hbase git commit: HBASE-14902 Revert some of the stringency recently introduced by checkstyle tightening; RETRY changing javadoc indent from 4 to 2

Posted by jm...@apache.org.
HBASE-14902 Revert some of the stringency recently introduced by checkstyle tightening; RETRY changing javadoc indent from 4 to 2


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

Branch: refs/heads/trunk
Commit: 72d32cc96b6cda7d7dc17cf3dc88235b01534d43
Parents: 46303df
Author: stack <st...@apache.org>
Authored: Tue Jan 5 14:45:05 2016 -0800
Committer: stack <st...@apache.org>
Committed: Tue Jan 5 14:45:05 2016 -0800

----------------------------------------------------------------------
 hbase-checkstyle/src/main/resources/hbase/checkstyle.xml | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/72d32cc9/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
index e7272c5..6095d99 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
@@ -77,7 +77,9 @@
 
     <!-- Javadoc Checks
     http://checkstyle.sourceforge.net/config_javadoc.html -->
-    <module name="JavadocTagContinuationIndentation"/>
+    <module name="JavadocTagContinuationIndentation">
+      <property name="offset" value="2"/>
+    </module>
     <module name="NonEmptyAtclauseDescription"/>
 
     <!-- Miscellaneous Checks


[05/50] [abbrv] hbase git commit: HBASE-15011 turn off the jdk8 javadoc linter.

Posted by jm...@apache.org.
HBASE-15011 turn off the jdk8 javadoc linter.


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

Branch: refs/heads/trunk
Commit: 822fead744a308df7ae45da440047207841d7abc
Parents: 9b8895b
Author: Sean Busbey <bu...@apache.org>
Authored: Sat Dec 26 16:47:53 2015 -0600
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue Dec 29 11:00:00 2015 -0600

----------------------------------------------------------------------
 pom.xml | 27 ++++++++++++++++++++++++++-
 1 file changed, 26 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/822fead7/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 08a1bf0..897b41c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -517,6 +517,11 @@
       <plugins>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
+           <artifactId>maven-javadoc-plugin</artifactId>
+           <version>${javadoc.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-remote-resources-plugin</artifactId>
           <version>1.5</version>
         </plugin>
@@ -1792,6 +1797,26 @@
 
   -->
   <profiles>
+    <profile>
+      <id>build-with-jdk8</id>
+      <activation>
+        <jdk>1.8</jdk>
+      </activation>
+      <build>
+        <pluginManagement>
+          <plugins>
+            <plugin>
+              <groupId>org.apache.maven.plugins</groupId>
+              <artifactId>maven-javadoc-plugin</artifactId>
+              <configuration>
+                <!-- TODO HBASE-15041 clean up our javadocs so jdk8 linter can be used -->
+                <additionalparam>-Xdoclint:none</additionalparam>
+              </configuration>
+            </plugin>
+          </plugins>
+        </pluginManagement>
+      </build>
+    </profile>
     <!-- profile activated by the Jenkins patch testing job -->
     <profile>
       <id>jenkins.patch</id>
@@ -2780,7 +2805,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-javadoc-plugin</artifactId>
-        <version>2.10.3</version>
+        <version>${javadoc.version}</version>
         <reportSets>
           <!-- Dev API -->
           <reportSet>


[04/50] [abbrv] hbase git commit: HBASE-15043 region_status.rb broken with TypeError: no public constructors for Java::OrgApacheHadoopHbaseClient::HBaseAdmin (Samir Ahmic)

Posted by jm...@apache.org.
HBASE-15043 region_status.rb broken with TypeError: no public constructors for Java::OrgApacheHadoopHbaseClient::HBaseAdmin (Samir Ahmic)


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

Branch: refs/heads/trunk
Commit: 9b8895ba295ee66fefda115f30ab727590e487e8
Parents: 1e4992c
Author: tedyu <yu...@gmail.com>
Authored: Tue Dec 29 07:06:43 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Dec 29 07:06:43 2015 -0800

----------------------------------------------------------------------
 bin/draining_servers.rb             |  5 ++++-
 bin/region_status.rb                |  7 ++++---
 bin/replication/copy_tables_desc.rb | 12 +++++++++---
 bin/shutdown_regionserver.rb        |  6 +++++-
 4 files changed, 22 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9b8895ba/bin/draining_servers.rb
----------------------------------------------------------------------
diff --git a/bin/draining_servers.rb b/bin/draining_servers.rb
index 45c9694..6849605 100644
--- a/bin/draining_servers.rb
+++ b/bin/draining_servers.rb
@@ -22,6 +22,7 @@ require 'optparse'
 include Java
 
 import org.apache.hadoop.hbase.HBaseConfiguration
+import org.apache.hadoop.hbase.client.ConnectionFactory
 import org.apache.hadoop.hbase.client.HBaseAdmin
 import org.apache.hadoop.hbase.zookeeper.ZKUtil
 import org.apache.commons.logging.Log
@@ -60,6 +61,7 @@ end
 
 def getServerNames(hostOrServers, config)
   ret = []
+  connection = ConnectionFactory.createConnection(config)
   
   for hostOrServer in hostOrServers
     # check whether it is already serverName. No need to connect to cluster
@@ -67,7 +69,7 @@ def getServerNames(hostOrServers, config)
     if parts.size() == 3
       ret << hostOrServer
     else 
-      admin = HBaseAdmin.new(config) if not admin
+      admin = connection.getAdmin() if not admin
       servers = getServers(admin)
 
       hostOrServer = hostOrServer.gsub(/:/, ",")
@@ -78,6 +80,7 @@ def getServerNames(hostOrServers, config)
   end
   
   admin.close() if admin
+  connection.close()
   return ret
 end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b8895ba/bin/region_status.rb
----------------------------------------------------------------------
diff --git a/bin/region_status.rb b/bin/region_status.rb
index a0f9d1b..55bc672 100644
--- a/bin/region_status.rb
+++ b/bin/region_status.rb
@@ -67,12 +67,11 @@ org.apache.log4j.Logger.getLogger("org.apache.hadoop.hbase").setLevel(log_level)
 config = HBaseConfiguration.create
 config.set 'fs.defaultFS', config.get(HConstants::HBASE_DIR)
 connection = ConnectionFactory.createConnection(config)
-
 # wait until the master is running
 admin = nil
 while true
   begin
-    admin = HBaseAdmin.new config
+    admin = connection.getAdmin()
     break
   rescue MasterNotRunningException => e
     print 'Waiting for master to start...\n'
@@ -119,7 +118,7 @@ while iter.hasNext
     # Gone too far, break
     break
   end
-  region = HRegionInfo.getHRegionInfo result
+  region = MetaTableAccessor::getHRegionInfo(result)
   if not region.isOffline
     # only include regions that should be online
     meta_count += 1
@@ -151,5 +150,7 @@ while true
     break
   end
 end
+admin.close()
+connection.close()
 
 exit server_count == meta_count ? 0 : 1

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b8895ba/bin/replication/copy_tables_desc.rb
----------------------------------------------------------------------
diff --git a/bin/replication/copy_tables_desc.rb b/bin/replication/copy_tables_desc.rb
index bc70031..8a6c670 100644
--- a/bin/replication/copy_tables_desc.rb
+++ b/bin/replication/copy_tables_desc.rb
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.EmptyWatcher
 import org.apache.hadoop.hbase.client.HBaseAdmin
 import org.apache.hadoop.hbase.HTableDescriptor
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hbase.client.ConnectionFactory
 
 # Name of this script
 NAME = "copy_tables_desc"
@@ -56,18 +57,23 @@ c1.set(HConstants::ZOOKEEPER_QUORUM, parts1[0])
 c1.set("hbase.zookeeper.property.clientPort", parts1[1])
 c1.set(HConstants::ZOOKEEPER_ZNODE_PARENT, parts1[2])
 
-admin1 = HBaseAdmin.new(c1)
+connection1 = ConnectionFactory.createConnection(c1)
+admin1 = connection1.getAdmin()
 
 c2 = HBaseConfiguration.create()
 c2.set(HConstants::ZOOKEEPER_QUORUM, parts2[0])
 c2.set("hbase.zookeeper.property.clientPort", parts2[1])
 c2.set(HConstants::ZOOKEEPER_ZNODE_PARENT, parts2[2])
 
-admin2 = HBaseAdmin.new(c2)
+connection2 = ConnectionFactory.createConnection(c2)
+admin2 = connection2.getAdmin()
 
 for t in admin1.listTables()
   admin2.createTable(t)
 end
 
-
 puts "All descriptions were copied"
+admin1.close()
+admin2.close()
+connection1.close()
+connection2.close()

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b8895ba/bin/shutdown_regionserver.rb
----------------------------------------------------------------------
diff --git a/bin/shutdown_regionserver.rb b/bin/shutdown_regionserver.rb
index 9287e5b..608248f 100644
--- a/bin/shutdown_regionserver.rb
+++ b/bin/shutdown_regionserver.rb
@@ -25,6 +25,7 @@
 include Java
 import org.apache.hadoop.hbase.HBaseConfiguration
 import org.apache.hadoop.hbase.client.HBaseAdmin
+import org.apache.hadoop.hbase.client.ConnectionFactory
 
 def usage(msg=nil)
   $stderr.puts 'Usage: shutdown_regionserver.rb <host:port>..'
@@ -41,8 +42,9 @@ ARGV.each do |x|
 end
 
 config = HBaseConfiguration.create()
+connection = ConnectionFactory.createConnection(config)
 begin
-  admin = HBaseAdmin.new(config)
+  admin = connection.getAdmin()
 rescue
   abort "Error: Couldn't instantiate HBaseAdmin"
 end
@@ -50,3 +52,5 @@ end
 ARGV.each do |hostport|
   admin.stopRegionServer(hostport)
 end
+admin.close()
+connection.close()


[46/50] [abbrv] hbase git commit: HBASE-15087 Fix hbase-common findbugs complaints

Posted by jm...@apache.org.
HBASE-15087 Fix hbase-common findbugs complaints


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

Branch: refs/heads/trunk
Commit: 83c506d9d49e900f3d4d5feca2ded135fb19b827
Parents: 8ee9158
Author: stack <st...@apache.org>
Authored: Tue Jan 12 00:30:29 2016 -0800
Committer: stack <st...@apache.org>
Committed: Tue Jan 12 00:30:42 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/CellComparator.java | 33 ++++++++++++++------
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  2 ++
 .../JitterScheduledThreadPoolExecutorImpl.java  | 14 ++++++++-
 .../apache/hadoop/hbase/OffheapKeyValue.java    |  4 +--
 .../org/apache/hadoop/hbase/ProcedureInfo.java  |  4 ++-
 .../hadoop/hbase/types/CopyOnWriteArrayMap.java | 10 +++++-
 .../hadoop/hbase/util/ByteBufferUtils.java      | 32 +++++++++++++++++--
 .../java/org/apache/hadoop/hbase/util/DNS.java  |  4 ++-
 .../hadoop/hbase/util/DynamicClassLoader.java   |  4 ++-
 .../apache/hadoop/hbase/util/UnsafeAccess.java  |  5 +--
 .../hadoop/hbase/util/TestByteBufferUtils.java  |  9 ++++++
 11 files changed, 100 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
index b179963..d869b3e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
@@ -171,9 +171,14 @@ public class CellComparator implements Comparator<Cell>, Serializable {
           right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
     }
     if (right instanceof ByteBufferedCell) {
-      return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getFamilyByteBuffer(),
-          ((ByteBufferedCell) right).getFamilyPosition(), right.getFamilyLength(),
-          left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength()));
+      // Notice how we flip the order of the compare here. We used to negate the return value but
+      // see what FindBugs says
+      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
+      // It suggest flipping the order to get same effect and 'safer'.
+      return ByteBufferUtils.compareTo(
+          left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+          ((ByteBufferedCell)right).getFamilyByteBuffer(),
+          ((ByteBufferedCell)right).getFamilyPosition(), right.getFamilyLength());
     }
     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
@@ -210,10 +215,14 @@ public class CellComparator implements Comparator<Cell>, Serializable {
           right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
     }
     if (right instanceof ByteBufferedCell) {
-      return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getQualifierByteBuffer(),
-          ((ByteBufferedCell) right).getQualifierPosition(),
-          right.getQualifierLength(), left.getQualifierArray(), left.getQualifierOffset(),
-          left.getQualifierLength()));
+      // Notice how we flip the order of the compare here. We used to negate the return value but
+      // see what FindBugs says
+      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
+      // It suggest flipping the order to get same effect and 'safer'.
+      return ByteBufferUtils.compareTo(left.getQualifierArray(),
+          left.getQualifierOffset(), left.getQualifierLength(),
+          ((ByteBufferedCell)right).getQualifierByteBuffer(),
+          ((ByteBufferedCell)right).getQualifierPosition(), right.getQualifierLength());
     }
     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
@@ -331,9 +340,13 @@ public class CellComparator implements Comparator<Cell>, Serializable {
           right.getRowArray(), right.getRowOffset(), right.getRowLength());
     }
     if (right instanceof ByteBufferedCell) {
-      return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getRowByteBuffer(),
-          ((ByteBufferedCell) right).getRowPosition(), right.getRowLength(),
-          left.getRowArray(), left.getRowOffset(), left.getRowLength()));
+      // Notice how we flip the order of the compare here. We used to negate the return value but
+      // see what FindBugs says
+      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
+      // It suggest flipping the order to get same effect and 'safer'.
+      return ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
+          ((ByteBufferedCell)right).getRowByteBuffer(),
+          ((ByteBufferedCell)right).getRowPosition(), right.getRowLength());
     }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
         right.getRowArray(), right.getRowOffset(), right.getRowLength());

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 1ec6afa..1b38b56 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -819,6 +819,8 @@ public final class CellUtil {
     }
 
     @Override
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IT_NO_SUCH_ELEMENT",
+      justification="Intentional")
     public Tag next() {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/JitterScheduledThreadPoolExecutorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/JitterScheduledThreadPoolExecutorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/JitterScheduledThreadPoolExecutorImpl.java
index 7e7239e..c330fa7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/JitterScheduledThreadPoolExecutorImpl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/JitterScheduledThreadPoolExecutorImpl.java
@@ -93,6 +93,19 @@ public class JitterScheduledThreadPoolExecutorImpl extends ScheduledThreadPoolEx
     }
 
     @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      }
+      return obj instanceof Delayed? compareTo((Delayed)obj) == 0: false;
+    }
+
+    @Override
+    public int hashCode() {
+      return this.wrapped.hashCode();
+    }
+
+    @Override
     public void run() {
       wrapped.run();
     }
@@ -123,5 +136,4 @@ public class JitterScheduledThreadPoolExecutorImpl extends ScheduledThreadPoolEx
       return wrapped.get(timeout, unit);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
index ced1595..d060b02 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
@@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.util.ClassSize;
  * memory.
  */
 @InterfaceAudience.Private
-public class OffheapKeyValue extends ByteBufferedCell implements HeapSize, Cloneable,
-    SettableSequenceId, Streamable {
+public class OffheapKeyValue extends ByteBufferedCell
+  implements HeapSize, SettableSequenceId, Streamable {
 
   protected final ByteBuffer buf;
   protected final int offset;

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
index 845a536..b7ea47e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.NonceKey;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class ProcedureInfo {
+public class ProcedureInfo implements Cloneable {
   private final long procId;
   private final String procName;
   private final String procOwner;
@@ -72,6 +72,8 @@ public class ProcedureInfo {
     this.result = result;
   }
 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="CN_IDIOM_NO_SUPER_CALL",
+      justification="Intentional; calling super class clone doesn't make sense here.")
   public ProcedureInfo clone() {
     return new ProcedureInfo(
       procId, procName, procOwner, procState, parentId, exception, lastUpdate, startTime, result);

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/types/CopyOnWriteArrayMap.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/CopyOnWriteArrayMap.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/CopyOnWriteArrayMap.java
index 41056b2..8de39ae 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/CopyOnWriteArrayMap.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/CopyOnWriteArrayMap.java
@@ -27,6 +27,7 @@ import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.NavigableSet;
+import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.concurrent.ConcurrentNavigableMap;
@@ -693,8 +694,10 @@ public class CopyOnWriteArrayMap<K, V> extends AbstractMap<K, V>
     }
 
     @Override
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_ALWAYS_FALSE",
+      justification="Intentional")
     public boolean equals(Object o) {
-      return false;
+      return false; // FindBugs: Causes EQ_ALWAYS_FALSE. Suppressed.
     }
 
     @Override
@@ -771,7 +774,12 @@ public class CopyOnWriteArrayMap<K, V> extends AbstractMap<K, V>
     }
 
     @Override
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IT_NO_SUCH_ELEMENT",
+      justification="Intentional")
     public Entry<K, V> next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
       return holder.entries[index++];
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 62173c2..99e798a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -38,6 +38,7 @@ import sun.nio.ch.DirectBuffer;
  * Utility functions for working with byte buffers, such as reading/writing
  * variable-length long numbers.
  */
+@SuppressWarnings("restriction")
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public final class ByteBufferUtils {
@@ -616,6 +617,33 @@ public final class ByteBufferUtils {
     return compareTo(buf1, o1, l1, buf2, o2, l2) == 0;
   }
 
+  public static int compareTo(byte [] buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
+    // This method is nearly same as the compareTo that follows but hard sharing code given
+    // byte array and bytebuffer types and this is a hot code path
+    if (UNSAFE_UNALIGNED) {
+      long offset2Adj;
+      Object refObj2 = null;
+      if (buf2.isDirect()) {
+        offset2Adj = o2 + ((DirectBuffer)buf2).address();
+      } else {
+        offset2Adj = o2 + buf2.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
+        refObj2 = buf2.array();
+      }
+      return compareToUnsafe(buf1, o1 + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, l1,
+          refObj2, offset2Adj, l2);
+    }
+    int end1 = o1 + l1;
+    int end2 = o2 + l2;
+    for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
+      int a = buf1[i] & 0xFF;
+      int b = buf2.get(i) & 0xFF;
+      if (a != b) {
+        return a - b;
+      }
+    }
+    return l1 - l2;
+  }
+
   public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
     if (UNSAFE_UNALIGNED) {
       long offset1Adj;
@@ -626,8 +654,8 @@ public final class ByteBufferUtils {
         offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
         refObj1 = buf1.array();
       }
-      return compareToUnsafe(refObj1, offset1Adj, l1, buf2, o2
-          + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, l2);
+      return compareToUnsafe(refObj1, offset1Adj, l1,
+          buf2, o2 + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, l2);
     }
     int end1 = o1 + l1;
     int end2 = o2 + l2;

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java
index d105a34..4b9e87f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DNS.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  * Wrapper around Hadoop's DNS class to hide reflection.
  */
 @InterfaceAudience.Private
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
+  justification="If exception, presume HAS_NEW_DNS_GET_DEFAULT_HOST_API false")
 public final class DNS {
   private static boolean HAS_NEW_DNS_GET_DEFAULT_HOST_API;
   private static Method GET_DEFAULT_HOST_METHOD;
@@ -35,7 +37,7 @@ public final class DNS {
           .getMethod("getDefaultHost", String.class, String.class, boolean.class);
       HAS_NEW_DNS_GET_DEFAULT_HOST_API = true;
     } catch (Exception e) {
-      HAS_NEW_DNS_GET_DEFAULT_HOST_API = false;
+      HAS_NEW_DNS_GET_DEFAULT_HOST_API = false; // FindBugs: Causes REC_CATCH_EXCEPTION. Suppressed
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java
index 2d5eb5d..214c917 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DynamicClassLoader.java
@@ -99,7 +99,9 @@ public class DynamicClassLoader extends ClassLoaderBase {
     }
   }
 
-  private void initTempDir(final Configuration conf) {
+  // FindBugs: Making synchronized to avoid IS2_INCONSISTENT_SYNC complaints about
+  // remoteDirFs and jarModifiedTime being part synchronized protected.
+  private synchronized void initTempDir(final Configuration conf) {
     jarModifiedTime = new HashMap<String, Long>();
     String localDirPath = conf.get(
       LOCAL_DIR_KEY, DEFAULT_LOCAL_DIR) + DYNAMIC_JARS_DIR;

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
index e72c9f0..af2632b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
@@ -34,6 +34,8 @@ import sun.nio.ch.DirectBuffer;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
+  justification="If exception, presume unaligned")
 public final class UnsafeAccess {
 
   private static final Log LOG = LogFactory.getLog(UnsafeAccess.class);
@@ -51,7 +53,6 @@ public final class UnsafeAccess {
   // copyMemory method. A limit is imposed to allow for safepoint polling
   // during a large copy
   static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L;
-
   static {
     theUnsafe = (Unsafe) AccessController.doPrivileged(new PrivilegedAction<Object>() {
       @Override
@@ -76,7 +77,7 @@ public final class UnsafeAccess {
         m.setAccessible(true);
         unaligned = (boolean) m.invoke(null);
       } catch (Exception e) {
-        unaligned = false;
+        unaligned = false; // FindBugs: Causes REC_CATCH_EXCEPTION. Suppressed.
       }
     } else{
       BYTE_ARRAY_BASE_OFFSET = -1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/83c506d9/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
index 2403c82..8ef07d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java
@@ -387,6 +387,15 @@ public class TestByteBufferUtils {
     assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) < 0);
     bb2.put(6, (byte) 4);
     assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) > 0);
+    // Assert reverse comparing BB and bytearray works.
+    ByteBuffer bb3 = ByteBuffer.allocate(135);
+    fillBB(bb3, (byte)0);
+    byte[] b3 = new byte[135];
+    fillArray(b3, (byte)1);
+    int result = ByteBufferUtils.compareTo(b3, 0, b3.length, bb3, 0, bb3.remaining());
+    assertTrue(result > 0);
+    result = ByteBufferUtils.compareTo(bb3, 0, bb3.remaining(), b3, 0, b3.length);
+    assertTrue(result < 0);
   }
 
   private static void fillBB(ByteBuffer bb, byte b) {


[41/50] [abbrv] hbase git commit: HBASE-15057 local-master-backup.sh doesn't start HMaster correctly (Samir Ahmic)

Posted by jm...@apache.org.
HBASE-15057 local-master-backup.sh doesn't start HMaster correctly (Samir Ahmic)


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

Branch: refs/heads/trunk
Commit: c2f67a053ebe687773e5e35aa7c060de964dbb68
Parents: 4a26644
Author: tedyu <yu...@gmail.com>
Authored: Fri Jan 8 15:58:01 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 8 15:58:01 2016 -0800

----------------------------------------------------------------------
 bin/local-master-backup.sh | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c2f67a05/bin/local-master-backup.sh
----------------------------------------------------------------------
diff --git a/bin/local-master-backup.sh b/bin/local-master-backup.sh
index bd265ab..cc2dc56 100755
--- a/bin/local-master-backup.sh
+++ b/bin/local-master-backup.sh
@@ -40,6 +40,7 @@ run_master () {
   DN=$2
   export HBASE_IDENT_STRING="$USER-$DN"
   HBASE_MASTER_ARGS="\
+    -D hbase.master.port=`expr 16000 + $DN` \
     -D hbase.master.info.port=`expr 16010 + $DN` \
     -D hbase.regionserver.port=`expr 16020 + $DN` \
     -D hbase.regionserver.info.port=`expr 16030 + $DN` \