You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2016/11/01 19:49:41 UTC

[17/50] [abbrv] hbase git commit: HBASE-16810 HBase Balancer throws ArrayIndexOutOfBoundsException when regionservers are in /hbase/draining znode and unloaded (David Pope)

HBASE-16810 HBase Balancer throws ArrayIndexOutOfBoundsException when regionservers are in /hbase/draining znode and unloaded (David Pope)


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

Branch: refs/heads/branch-1
Commit: ca581874b9d39cef5d8f142f25ab77fbdef98237
Parents: 27398ea
Author: tedyu <yu...@gmail.com>
Authored: Thu Oct 13 08:42:49 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Oct 13 08:42:49 2016 -0700

----------------------------------------------------------------------
 .../master/balancer/StochasticLoadBalancer.java |   3 +-
 .../hbase/master/MockNoopMasterServices.java    | 292 +++++++++++++++++++
 .../hbase/master/balancer/BalancerTestBase.java |  80 +++++
 .../balancer/TestStochasticLoadBalancer.java    |  16 +
 4 files changed, 389 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index 181990b..d497d42 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action.Type;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.AssignRegionAction;
@@ -1238,7 +1237,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
         if (index < 0) {
           cost += 1;
         } else {
-          cost += (1 - cluster.getLocalityOfRegion(i, index));
+          cost += (1 - cluster.getLocalityOfRegion(i, serverIndex));
         }
       }
       return scale(0, max, cost);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
new file mode 100644
index 0000000..657d8e2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -0,0 +1,292 @@
+/**
+ * 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 com.google.protobuf.Service;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
+import java.io.IOException;
+import java.util.List;
+
+public class MockNoopMasterServices implements MasterServices, Server {
+  @Override
+  public SnapshotManager getSnapshotManager() {
+    return null;
+  }
+
+  @Override
+  public MasterProcedureManagerHost getMasterProcedureManagerHost() {
+    return null;
+  }
+
+  @Override
+  public AssignmentManager getAssignmentManager() {
+    return null;
+  }
+
+  @Override
+  public MasterFileSystem getMasterFileSystem() {
+    return null;
+  }
+
+  @Override
+  public ServerManager getServerManager() {
+    return null;
+  }
+
+  @Override
+  public ExecutorService getExecutorService() {
+    return null;
+  }
+
+  @Override
+  public TableLockManager getTableLockManager() {
+    return null;
+  }
+
+  @Override
+  public MasterCoprocessorHost getMasterCoprocessorHost() {
+    return null;
+  }
+
+  @Override
+  public TableNamespaceManager getTableNamespaceManager() {
+    return null;
+  }
+
+  @Override
+  public MasterQuotaManager getMasterQuotaManager() {
+    return null;
+  }
+
+  @Override
+  public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return null;
+  }
+
+  @Override
+  public void checkTableModifiable(TableName tableName) throws IOException, TableNotFoundException, TableNotDisabledException {
+
+  }
+
+  @Override
+  public long createTable(HTableDescriptor desc, byte[][] splitKeys, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long createSystemTable(HTableDescriptor hTableDescriptor) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long deleteTable(TableName tableName, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void truncateTable(TableName tableName, boolean preserveSplits, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void modifyTable(TableName tableName, HTableDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public long enableTable(TableName tableName, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long disableTable(TableName tableName, long nonceGroup, long nonce) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void addColumn(TableName tableName, HColumnDescriptor column, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void modifyColumn(TableName tableName, HColumnDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void deleteColumn(TableName tableName, byte[] columnName, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public TableDescriptors getTableDescriptors() {
+    return null;
+  }
+
+  @Override
+  public boolean isServerCrashProcessingEnabled() {
+    return false;
+  }
+
+  @Override
+  public boolean registerService(Service instance) {
+    return false;
+  }
+
+  @Override
+  public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, boolean forcible, User user) throws IOException {
+
+  }
+
+  @Override
+  public boolean isInitialized() {
+    return false;
+  }
+
+  @Override
+  public void createNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void createNamespaceSync(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public void deleteNamespace(String name, long nonceGroup, long nonce) throws IOException {
+
+  }
+
+  @Override
+  public boolean isInMaintenanceMode() {
+    return false;
+  }
+
+  @Override
+  public boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException {
+    return false;
+  }
+
+  @Override
+  public List<ProcedureInfo> listProcedures() throws IOException {
+    return null;
+  }
+
+  @Override
+  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return null;
+  }
+
+  @Override
+  public ZooKeeperWatcher getZooKeeper() {
+    return null;
+  }
+
+  @Override
+  public ClusterConnection getConnection() {
+    return null;
+  }
+
+  @Override
+  public MetaTableLocator getMetaTableLocator() {
+    return null;
+  }
+
+  @Override
+  public ServerName getServerName() {
+    return null;
+  }
+
+  @Override
+  public CoordinatedStateManager getCoordinatedStateManager() {
+    return null;
+  }
+
+  @Override
+  public ChoreService getChoreService() {
+    return null;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+
+  }
+
+  @Override
+  public boolean isAborted() {
+    return false;
+  }
+
+  @Override
+  public void stop(String why) {
+
+  }
+
+  @Override
+  public boolean isStopped() {
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
index 7ae0133..5746e3f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -150,6 +151,85 @@ public class BalancerTestBase {
 
   };
 
+  /**
+   * Data set for testLocalityCost:
+   *
+   * [test][regions][0] = [serverIndex] -> number of regions
+   * [test][regions][regionIndex+1] = {server hosting region, locality percentage, datanodes}
+   *
+   * For each [test], there is a list of cluster config information grouped by [regions].
+   * - [0] - the first element of the [regions] list is a list of servers with the value
+   *         indicating the number of regions it hosts.
+   * - [regionIndex+1] - the remaining elements of the array are regions, where the index value
+   *         is 1 greater than the regionIndex.  This element holds an array that identifies:
+   *     [0] - the serverIndex of the server hosting this region
+   *     [1] - the locality percentage returned by getLocalityOfRegion(region, server) when the
+   *           server is hosting both region and the hdfs blocks.
+   *     [.] - the serverIndex of servers hosting the hdfs blocks, where a value of -1 indicates
+   *         a dfs server not in the list of region servers.
+   */
+  protected int[][][] clusterRegionLocationMocks = new int[][][]{
+      // Test 1: Basic region placement with 1 region server not hosting dfs block
+      //     Locality Calculation:
+      //        region[0] = 1 - 80/100 = (.2)  - server[2] hosts both the region and dfs blocks
+      //        region[1] = 1.0                - server[0] only hosts the region, not dfs blocks
+      //        region[2] = 1 - 70/100 = (.3)  - server[1] hosts both the region and dfs blocks
+      //
+      //      RESULT = 0.2 + 1.0 + 0.3 / 3.0 (3.0 is max value)
+      //             = 1.5 / 3.0
+      //             = 0.5
+      new int[][]{
+          new int[]{1, 1, 1},         // 3 region servers with 1 region each
+          new int[]{2, 80, 1, 2, 0},  // region[0] on server[2] w/ 80% locality
+          new int[]{0, 50, 1, 2},     // region[1] on server[0] w/ 50% , but no local dfs blocks
+          new int[]{1, 70, 2, 0, 1},  // region[2] on server[1] w/ 70% locality
+      },
+
+      // Test 2: Sames as Test 1, but the last region has a datanode that isn't a region server
+      new int[][]{
+          new int[]{1, 1, 1},
+          new int[]{2, 80, 1, 2, 0},
+          new int[]{0, 50, 1, 2},
+          new int[]{1, 70, -1, 2, 0, 1},  // the first region location is not on a region server
+      },
+  };
+
+  // This mock allows us to test the LocalityCostFunction
+  protected class MockCluster extends BaseLoadBalancer.Cluster {
+
+    protected int[][] localityValue = null;   // [region][server] = percent of blocks
+
+    protected MockCluster(int[][] regions) {
+
+      // regions[0] is an array where index = serverIndex an value = number of regions
+      super(mockClusterServers(regions[0], 1), null, null, null);
+
+      localityValue = new int[regions.length-1][];
+      // the remaining elements in the regions array contain values for:
+      //   [0] - the serverIndex of the server hosting this region
+      //   [1] - the locality percentage (in whole numbers) for the hosting region server
+      //   [.] - a list of servers hosting dfs blocks for the region (-1 means its not one
+      //         of our region servers.
+      for (int i = 1; i < regions.length; i++){
+        int regionIndex = i - 1;
+        int serverIndex = regions[i][0];
+        int locality = regions[i][1];
+        int[] locations = Arrays.copyOfRange(regions[i], 2, regions[i].length);
+
+        regionIndexToServerIndex[regionIndex] = serverIndex;
+        localityValue[regionIndex] = new int[servers.length];
+        localityValue[regionIndex][serverIndex] = (locality > 100)? locality % 100 : locality;
+        regionLocations[regionIndex] = locations;
+      }
+    }
+
+    @Override
+    float getLocalityOfRegion(int region, int server) {
+      // convert the locality percentage to a fraction
+      return localityValue[region][server] / 100.0f;
+    }
+  }
+
   // This class is introduced because IP to rack resolution can be lengthy.
   public static class MockMapping implements DNSToSwitchMapping {
     public MockMapping(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ca581874/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
index 9caf264..094687b 100644
--- 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
@@ -44,6 +44,7 @@ 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.MockNoopMasterServices;
 import org.apache.hadoop.hbase.master.RackManager;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
@@ -133,7 +134,22 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
         returnServer(entry.getKey());
       }
     }
+  }
+
+  @Test
+  public void testLocalityCost() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    MockNoopMasterServices master = new MockNoopMasterServices();
+    StochasticLoadBalancer.CostFunction
+        costFunction = new StochasticLoadBalancer.LocalityCostFunction(conf, master);
 
+    for (int[][] clusterRegionLocations : clusterRegionLocationMocks) {
+      MockCluster cluster = new MockCluster(clusterRegionLocations);
+      costFunction.init(cluster);
+      double cost = costFunction.cost();
+
+      assertEquals(0.5f, cost, 0.001);
+    }
   }
 
   @Test