You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2012/04/07 00:11:18 UTC

svn commit: r1310607 [1/2] - in /hbase/branches/0.89-fb: bin/ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/regionserver/ src/main/java/org/apache/hadoop/hbase/util/ src/test/...

Author: mbautin
Date: Fri Apr  6 22:11:17 2012
New Revision: 1310607

URL: http://svn.apache.org/viewvc?rev=1310607&view=rev
Log:
[master] Assignment plan improvement and region assignment verification report

Summary:
The previous diff D407485, Integrate the favoredNodes into the new assignment manager, enabled the Master to assign regions based on the assignment plan. This diff is to improvement the assignment plan and to provide a tool to verify the region assignment.

1) Improvement the assignment plan for the new created tables:
Each new created table has one assignment domain. Currently each new table will have its own assignment domain by shuffling all the region servers based on the table name.

The assignment domain will be easy to extend for multi tenancy.

The primary region server is placed in the round robin way, fairly across different racks and hosts. Also the secondary and tertiary region server will be placed by some randomness. The randomness can make sure that the regions, shared with the same primary region server, have a very high probability to run with different secondary region server and tertiary region server.

2) Improvement the assignment plan for the existing tables:
The region placement tool can take a snapshot for the current region assignment situation by scanning META and also can get a locality report from the file system. Based on these two information, the tool can place the primary region server for EACH TABLE to achieve the maximum table level locality, the minimum region movement and slot-based balancing.

In addition, the tool use the same way to place the secondary and tertiary region server as the new created table case.

3) Region assignment verification report
The region placement tool can also generate the region assignment report for each table. Here is the sample report from my dev cluster.

Test Plan:
1) run all the unit tests
2) tested on dev cluster
3) tested on migrate002 cluster

Reviewers: kannan, kranganathan

Reviewed By: kranganathan

CC: hbase-eng@

Differential Revision: https://phabricator.fb.com/D438425

Added:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentDomain.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RackManager.java
      - copied, changed from r1310606, hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RackManager.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionAssignmentSnapshot.java
Removed:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RackManager.java
Modified:
    hbase/branches/0.89-fb/bin/hbase
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerAddress.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentPlan.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionPlacement.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementPolicy.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java

Modified: hbase/branches/0.89-fb/bin/hbase
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/bin/hbase?rev=1310607&r1=1310606&r2=1310607&view=diff
==============================================================================
--- hbase/branches/0.89-fb/bin/hbase (original)
+++ hbase/branches/0.89-fb/bin/hbase Fri Apr  6 22:11:17 2012
@@ -77,6 +77,7 @@ if [ $# = 0 ]; then
   echo "  migrate          upgrade an hbase.rootdir"
   echo "  hbck             run the hbase 'fsck' tool"
   echo "  localityck	     run the hbase locality check tool"
+  echo "  region_placement run the hbase region placement tool"
   echo "  verify [-help]   verify that the cluster is working properly"
   echo " or"
   echo "  CLASSNAME        run the class named CLASSNAME"
@@ -264,6 +265,8 @@ elif [ "$COMMAND" = "hbck" ] ; then
   CLASS='org.apache.hadoop.hbase.client.HBaseFsck'
 elif [ "$COMMAND" = "localityck" ] ; then
   CLASS='org.apache.hadoop.hbase.client.HBaseLocalityCheck'
+elif [ "$COMMAND" = "region_placement" ] ; then
+  CLASS='org.apache.hadoop.hbase.master.RegionPlacement'
 elif [ "$COMMAND" = "verify" ] ; then
   CLASS='org.apache.hadoop.hbase.loadtest.LoadTester'
 elif [ "$COMMAND" = "zookeeper" ] ; then

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerAddress.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerAddress.java?rev=1310607&r1=1310606&r2=1310607&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerAddress.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HServerAddress.java Fri Apr  6 22:11:17 2012
@@ -34,10 +34,12 @@ import org.apache.hadoop.io.WritableComp
 public class HServerAddress implements WritableComparable<HServerAddress> {
   private InetSocketAddress address;
   private String stringValue;
+  private String hostAddress;
 
   public HServerAddress() {
     this.address = null;
     this.stringValue = null;
+    this.hostAddress = null;
   }
 
   /**
@@ -110,6 +112,9 @@ public class HServerAddress implements W
 
   /** @return Bind address */
   public String getBindAddress() {
+    if (this.hostAddress != null)
+      return hostAddress;
+
     final InetAddress addr = address.getAddress();
     if (addr != null) {
       return addr.getHostAddress();
@@ -121,7 +126,7 @@ public class HServerAddress implements W
   }
 
   private void checkBindAddressCanBeResolved() {
-    if (getBindAddress() == null) {
+    if ((this.hostAddress = getBindAddress()) == null) {
       throw new IllegalArgumentException("Could not resolve the"
           + " DNS name of " + stringValue);
     }

Added: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentDomain.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentDomain.java?rev=1310607&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentDomain.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentDomain.java Fri Apr  6 22:11:17 2012
@@ -0,0 +1,204 @@
+/**
+ * Copyright 2011 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.master;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
+
+public class AssignmentDomain {
+  protected static final Log LOG =
+    LogFactory.getLog(AssignmentDomain.class.getClass());
+  private Map<String, List<HServerAddress>> rackToRegionServerMap;
+  private List<String> uniqueRackList;
+  private RackManager rackManager;
+  private Map<HServerAddress, String> regionServerToRackMap;
+  private Random random;
+
+  public AssignmentDomain(Configuration conf) {
+    rackToRegionServerMap = new HashMap<String, List<HServerAddress>>();
+    regionServerToRackMap = new HashMap<HServerAddress, String>();
+    uniqueRackList = new ArrayList<String>();
+    rackManager = new RackManager(conf);
+    random = new Random();
+  }
+
+  /**
+   * Set the random seed
+   * @param seed
+   */
+  public void setRandomSeed(long seed) {
+    random.setSeed(seed);
+  }
+
+  /**
+   * Get the rack name in this domain for the server.
+   * @param server
+   * @return
+   */
+  public String getRack(HServerAddress server) {
+    if (server == null)
+      return null;
+    return regionServerToRackMap.get(server);
+  }
+
+  /**
+   * Get a random rack except for the current rack
+   * @param skipRackSet
+   * @return the random rack except for any Rack from the skipRackSet
+   * @throws IOException
+   */
+  public String getOneRandomRack(Set<String> skipRackSet) throws IOException {
+    if (skipRackSet == null || uniqueRackList.size() <= skipRackSet.size()) {
+      throw new IOException("Cannot randomly pick another random server");
+    }
+
+    String randomRack;
+    do {
+      int randomIndex = random.nextInt(this.uniqueRackList.size());
+      randomRack = this.uniqueRackList.get(randomIndex);
+    } while (skipRackSet.contains(randomRack));
+
+    return randomRack;
+  }
+
+  /**
+   * Get one random server from the rack
+   * @param rack
+   * @return
+   * @throws IOException
+   */
+  public HServerAddress getOneRandomServer(String rack) throws IOException {
+    return this.getOneRandomServer(rack, null);
+  }
+
+  /**
+   * Get a random server from the rack except for the servers in the skipServerSet
+   * @param skipServerSet
+   * @return the random server except for any servers from the skipServerSet
+   * @throws IOException
+   */
+  public HServerAddress getOneRandomServer(String rack,
+      Set<HServerAddress> skipServerSet) throws IOException {
+    if(rack == null) return null;
+    List<HServerAddress> serverList = this.rackToRegionServerMap.get(rack);
+    if (serverList == null) return null;
+
+    // Get a random server except for any servers from the skip set
+    if (skipServerSet != null && serverList.size() <= skipServerSet.size()) {
+      throw new IOException("Cannot randomly pick another random server");
+    }
+
+    HServerAddress randomServer;
+    do {
+      int randomIndex = random.nextInt(serverList.size());
+      randomServer = serverList.get(randomIndex);
+    } while (skipServerSet != null && skipServerSet.contains(randomServer));
+
+    return randomServer;
+  }
+
+  /**
+   * @return the total number of unique rack in the domain.
+   */
+  public int getTotalRackNum() {
+    return this.uniqueRackList.size();
+  }
+
+  /**
+   * Get the list of region severs in the rack
+   * @param rack
+   * @return the list of region severs in the rack
+   */
+  public List<HServerAddress> getServersFromRack(String rack) {
+    return this.rackToRegionServerMap.get(rack);
+  }
+
+  /**
+   * Add a server to the assignment domain
+   * @param server
+   */
+  public void addServer(HServerAddress server) {
+    // For a new server
+    String rackName = this.rackManager.getRack(server);
+    List<HServerAddress> serverList = this.rackToRegionServerMap.get(rackName);
+    if (serverList == null) {
+      serverList = new ArrayList<HServerAddress>();
+      // Add the current rack to the unique rack list
+      this.uniqueRackList.add(rackName);
+    }
+    if (!serverList.contains(server)) {
+      serverList.add(server);
+      this.rackToRegionServerMap.put(rackName, serverList);
+      this.regionServerToRackMap.put(server, rackName);
+    }
+  }
+
+  /**
+   * Add a list of servers to the assignment domain
+   * @param servers
+   */
+  public void addServers(List<HServerAddress> servers) {
+    for (HServerAddress server : servers) {
+      this.addServer(server);
+    }
+  }
+
+  /**
+   * Get the region server to rack map
+   */
+  public Map<HServerAddress, String> getRegionServerToRackMap() {
+    return this.regionServerToRackMap;
+  }
+
+  /**
+   * Get the rack to region server map
+   */
+  public Map<String, List<HServerAddress>> getRackToRegionServerMap() {
+    return this.rackToRegionServerMap;
+  }
+
+  /**
+   * @return true if there is no rack in the assignment domain
+   */
+  public boolean isEmpty() {
+    return uniqueRackList.isEmpty();
+  }
+
+  /**
+   * @return true if can place the favored nodes
+   */
+  public boolean canPlaceFavoredNodes() {
+    int serverSize = this.regionServerToRackMap.keySet().size();
+    if (serverSize < HConstants.FAVORED_NODES_NUM)
+      return false;
+    return true;
+  }
+}

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentPlan.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentPlan.java?rev=1310607&r1=1310606&r2=1310607&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentPlan.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentPlan.java Fri Apr  6 22:11:17 2012
@@ -29,8 +29,10 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -42,7 +44,8 @@ import org.apache.hadoop.io.Writable;
  */
 public class AssignmentPlan implements Writable{
   protected static final Log LOG = LogFactory.getLog(
-      AssignmentPlan.class);
+      AssignmentPlan.class.getName());
+
   private static final int VERSION = 1;
 
   /** the map between each region and its favored region server list */
@@ -65,6 +68,42 @@ public class AssignmentPlan implements W
   }
 
   /**
+   * Initialize the assignment plan with the existing primary region server map
+   * and the existing secondary/tertiary region server map
+   *
+   * if any regions cannot find the proper secondary / tertiary region server
+   * for whatever reason, just do NOT update the assignment plan for this region
+   * @param primaryRSMap
+   * @param secondaryAndTiteraryRSMap
+   */
+  public void initialize(Map<HRegionInfo, HServerAddress> primaryRSMap,
+      Map<HRegionInfo, Pair<HServerAddress, HServerAddress>> secondaryAndTertiaryRSMap) {
+    for (Map.Entry<HRegionInfo, Pair<HServerAddress, HServerAddress>> entry :
+      secondaryAndTertiaryRSMap.entrySet()) {
+      // Get the region info and their secondary/tertiary region server
+      HRegionInfo regionInfo = entry.getKey();
+      Pair<HServerAddress, HServerAddress> secondaryAndTertiaryPair =
+        entry.getValue();
+
+      // Get the primary region server
+      HServerAddress primaryRS = primaryRSMap.get(regionInfo);
+      if (primaryRS == null) {
+        LOG.error("No primary region server for region " +
+            regionInfo.getRegionNameAsString());
+        continue;
+      }
+
+      // Update the assignment plan with the favored nodes
+      List<HServerAddress> serverList = new ArrayList<HServerAddress>();
+      serverList.add(POSITION.PRIMARY.ordinal(), primaryRS);
+      serverList.add(POSITION.SECONDARY.ordinal(),
+          secondaryAndTertiaryPair.getFirst());
+      serverList.add(POSITION.TERTIARY.ordinal(),
+          secondaryAndTertiaryPair.getSecond());
+      this.updateAssignmentPlan(regionInfo, serverList);
+    }
+  }
+  /**
    * Add an assignment to the plan
    * @param region
    * @param servers
@@ -224,4 +263,18 @@ public class AssignmentPlan implements W
    }
    return true;
  }
+
+ public static AssignmentPlan.POSITION getFavoredServerPosition(
+     List<HServerAddress> favoredNodes, HServerAddress server) {
+   if (favoredNodes == null || server == null ||
+       favoredNodes.size() != HConstants.FAVORED_NODES_NUM) {
+     return null;
+   }
+   for (AssignmentPlan.POSITION p : AssignmentPlan.POSITION.values()) {
+     if (favoredNodes.get(p.ordinal()).equals(server)) {
+       return p;
+     }
+   }
+   return null;
+ }
 }

Added: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java?rev=1310607&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/AssignmentVerificationReport.java Fri Apr  6 22:11:17 2012
@@ -0,0 +1,307 @@
+/**
+ * Copyright 2012 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.master;
+
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+
+public class AssignmentVerificationReport {
+  protected static final Log LOG = LogFactory.getLog(
+      AssignmentVerificationReport.class.getName());
+
+  private String tableName = null;
+  private boolean enforceLocality = false;
+  private boolean isFilledUp = false;
+
+  private int totalRegions = 0;
+  private int totalRegionServers = 0;
+  // for unassigned regions
+  private List<HRegionInfo> unAssignedRegionsList =
+    new ArrayList<HRegionInfo>();
+
+  // For regions without valid favored nodes
+  private List<HRegionInfo> regionsWithoutValidFavoredNodes =
+    new ArrayList<HRegionInfo>();
+
+  // For regions not running on the favored nodes
+  private List<HRegionInfo> nonFavoredAssignedRegionList =
+    new ArrayList<HRegionInfo>();
+
+  // For regions running on the favored nodes
+  private int totalFavoredAssignments = 0;
+  private int[] favoredNodes = new int[HConstants.FAVORED_NODES_NUM];
+  private float[] favoredNodesLocalitySummary = new float[HConstants.FAVORED_NODES_NUM];
+  private float actualLocalitySummary = 0;
+
+  // For region balancing information
+  private int avgRegionsOnRS = 0;
+  private int maxRegionsOnRS = 0;
+  private int minRegionsOnRS = Integer.MAX_VALUE;
+  private Set<HServerAddress> mostLoadedRSSet =
+    new HashSet<HServerAddress>();
+  private Set<HServerAddress> leastLoadedRSSet =
+    new HashSet<HServerAddress>();
+
+  public void fillUp(String tableName, RegionAssignmentSnapshot snapshot,
+      Map<String, Map<String, Float>> regionLocalityMap) {
+    // Set the table name
+    this.tableName = tableName;
+
+    // Get all the regions for this table
+    List<HRegionInfo> regionInfoList =
+      snapshot.getTableToRegionMap().get(tableName);
+    // Get the total region num for the current table
+    this.totalRegions = regionInfoList.size();
+
+    // Get the existing assignment plan
+    AssignmentPlan plan = snapshot.getExistingAssignmentPlan();
+    // Get the region to region server mapping
+    Map<HRegionInfo, HServerAddress> currentAssignment =
+      snapshot.getRegionToRegionServerMap();
+    // Initialize the server to its hosing region counter map
+    Map<HServerAddress, Integer> serverToHostingRegionCounterMap =
+      new HashMap<HServerAddress, Integer>();
+
+
+    // Check the favored nodes and its locality information
+    // Also keep tracker of the most loaded and least loaded region servers
+    for (HRegionInfo region : regionInfoList) {
+      try {
+        HServerAddress currentRS = currentAssignment.get(region);
+        // Handle unassigned regions
+        if (currentRS == null) {
+          unAssignedRegionsList.add(region);
+          continue;
+        }
+
+        // Keep updating the server to is hosting region counter map
+        Integer hostRegionCounter = serverToHostingRegionCounterMap.get(currentRS);
+        if (hostRegionCounter == null) {
+          hostRegionCounter = new Integer(0);
+        }
+        hostRegionCounter = hostRegionCounter.intValue() + 1;
+        serverToHostingRegionCounterMap.put(currentRS, hostRegionCounter);
+
+        // Get the favored nodes from the assignment plan and verify it.
+        List<HServerAddress> favoredNodes = plan.getAssignment(region);
+        if (favoredNodes == null ||
+            favoredNodes.size() != HConstants.FAVORED_NODES_NUM) {
+          regionsWithoutValidFavoredNodes.add(region);
+          continue;
+        }
+
+        // Get the position of the current region server in the favored nodes list
+        AssignmentPlan.POSITION favoredNodePosition =
+          AssignmentPlan.getFavoredServerPosition(favoredNodes, currentRS);
+
+        // Handle the non favored assignment.
+        if (favoredNodePosition == null) {
+          nonFavoredAssignedRegionList.add(region);
+          continue;
+        }
+        // Increase the favored nodes assignment.
+        this.favoredNodes[favoredNodePosition.ordinal()]++;
+        totalFavoredAssignments++;
+
+        // Summary the locality information for each favored nodes
+        if (regionLocalityMap != null) {
+          // Set the enforce locality as true;
+          this.enforceLocality = true;
+
+          // Get the region degree locality map
+          Map<String, Float> regionDegreeLocalityMap =
+            regionLocalityMap.get(region.getEncodedName());
+          if (regionDegreeLocalityMap == null) {
+            continue; // ignore the region which doesn't have any store files.
+          }
+
+          // Get the locality summary for each favored nodes
+          for (AssignmentPlan.POSITION p : AssignmentPlan.POSITION.values()) {
+            HServerAddress favoredNode = favoredNodes.get(p.ordinal());
+            // Get the locality for the current favored nodes
+            Float locality =
+              regionDegreeLocalityMap.get(favoredNode.getHostname());
+            if (locality != null) {
+              this.favoredNodesLocalitySummary[p.ordinal()] += locality;
+            }
+          }
+
+
+          // Get the locality summary for the current region server
+          Float actualLocality =
+            regionDegreeLocalityMap.get(currentRS.getHostname());
+          if (actualLocality != null) {
+            this.actualLocalitySummary += actualLocality;
+          }
+        }
+      } catch (Exception e) {
+        LOG.error("Cannot verify the region assignment for region " +
+            ((region == null) ? " null " : region.getRegionNameAsString()) +
+            "becuase of " + e);
+      }
+    }
+
+    // Fill up the most loaded and least loaded region server information
+    for (Map.Entry<HServerAddress, Integer> entry :
+      serverToHostingRegionCounterMap.entrySet()) {
+      HServerAddress currentRS = entry.getKey();
+      int hostRegionCounter = entry.getValue().intValue();
+
+      // Update the most loaded region server list and maxRegionsOnRS
+      if (hostRegionCounter > this.maxRegionsOnRS) {
+        maxRegionsOnRS = hostRegionCounter;
+        this.mostLoadedRSSet.clear();
+        this.mostLoadedRSSet.add(currentRS);
+      } else if (hostRegionCounter == this.maxRegionsOnRS) {
+        this.mostLoadedRSSet.add(currentRS);
+      }
+
+      // Update the least loaded region server list and minRegionsOnRS
+      if (hostRegionCounter < this.minRegionsOnRS) {
+        this.minRegionsOnRS = hostRegionCounter;
+        this.leastLoadedRSSet.clear();
+        this.leastLoadedRSSet.add(currentRS);
+      } else if (hostRegionCounter == this.minRegionsOnRS) {
+        this.leastLoadedRSSet.add(currentRS);
+      }
+    }
+
+    // and total region servers
+    this.totalRegionServers = serverToHostingRegionCounterMap.keySet().size();
+    this.avgRegionsOnRS = (totalRegionServers == 0) ? 0 :
+      (totalRegions / totalRegionServers);
+
+    // Set the isFilledUp as true
+    isFilledUp = true;
+  }
+
+  public void print(boolean isDetailMode) {
+    if (!isFilledUp) {
+      System.err.println("[Error] Region assignment verfication report" +
+          "hasn't been filled up");
+    }
+    DecimalFormat df = new java.text.DecimalFormat( "#.##");
+
+    // Print some basic information
+    System.out.println("Region Assignment Verification for Table: " + tableName +
+        "\n\tTotal regions : " + totalRegions);
+
+    // Print the number of regions on each kinds of the favored nodes
+    System.out.println("\tTotal regions on favored nodes " +
+        totalFavoredAssignments);
+    for (AssignmentPlan.POSITION p : AssignmentPlan.POSITION.values()) {
+      System.out.println("\t\tTotal regions on "+ p.toString() +
+          " region servers: " + favoredNodes[p.ordinal()]);
+    }
+    // Print the number of regions in each kinds of invalid assignment
+    System.out.println("\tTotal unassigned regions: " +
+        unAssignedRegionsList.size());
+    if (isDetailMode) {
+      for (HRegionInfo region : unAssignedRegionsList) {
+        System.out.println("\t\t" + region.getRegionNameAsString());
+      }
+    }
+
+    System.out.println("\tTotal regions NOT on favored nodes: " +
+        nonFavoredAssignedRegionList.size());
+    if (isDetailMode) {
+      for (HRegionInfo region : nonFavoredAssignedRegionList) {
+        System.out.println("\t\t" + region.getRegionNameAsString());
+      }
+    }
+
+    System.out.println("\tTotal regions without favored nodes: " +
+        regionsWithoutValidFavoredNodes.size());
+    if (isDetailMode) {
+      for (HRegionInfo region : regionsWithoutValidFavoredNodes) {
+        System.out.println("\t\t" + region.getRegionNameAsString());
+      }
+    }
+
+    // Print the locality information if enabled
+    if (this.enforceLocality && totalRegions != 0) {
+      // Print the actual locality for this table
+      float actualLocality = 100 *
+        this.actualLocalitySummary / (float) totalRegions;
+      System.out.println("\n\tThe actual avg locality is " +
+          df.format(actualLocality) + " %");
+
+      // Print the expected locality if regions are placed on the each kinds of
+      // favored nodes
+      for (AssignmentPlan.POSITION p : AssignmentPlan.POSITION.values()) {
+        float avgLocality = 100 *
+          (favoredNodesLocalitySummary[p.ordinal()] / (float) totalRegions);
+        System.out.println("\t\tThe expected avg locality if all regions" +
+			" on the " + p.toString() + " region servers: "
+			+ df.format(avgLocality) + " %");
+      }
+    }
+
+    // Print the region balancing information
+    System.out.println("\n\tTotal hosting region servers: " +
+        totalRegionServers);
+    // Print the region balance information
+    if (totalRegionServers != 0) {
+      System.out.println("\tAvg regions/region server: " + avgRegionsOnRS +
+          ";\tMax regions/region server: " + maxRegionsOnRS +
+          ";\tMin regions/region server: " + minRegionsOnRS);
+
+      // Print the details about the most loaded region servers
+      System.out.println("\tThe number of the most loaded region servers: "
+          + mostLoadedRSSet.size());
+      if (isDetailMode) {
+        int i = 0;
+        for (HServerAddress addr : mostLoadedRSSet){
+          if ((i++) % 3 == 0) {
+            System.out.print("\n\t\t");
+          }
+          System.out.print(addr.getHostNameWithPort() + " ; ");
+        }
+        System.out.println("\n");
+      }
+
+      // Print the details about the least loaded region servers
+      System.out.println("\tThe number of the least loaded region servers: "
+          + leastLoadedRSSet.size());
+      if (isDetailMode) {
+        int i = 0;
+        for (HServerAddress addr : leastLoadedRSSet){
+          if ((i++) % 3 == 0) {
+            System.out.print("\n\t\t");
+          }
+          System.out.print(addr.getHostNameWithPort() + " ; ");
+        }
+        System.out.println();
+      }
+    }
+    System.out.println("==============================");
+  }
+}

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1310607&r1=1310606&r2=1310607&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Fri Apr  6 22:11:17 2012
@@ -31,10 +31,12 @@ import java.lang.reflect.Constructor;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Random;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.concurrent.ThreadFactory;
@@ -1360,8 +1362,17 @@ public class HMaster extends Thread impl
 
     AssignmentPlan assignmentPlan = null;
     if (this.shouldAssignRegionsWithFavoredNodes) {
+      // Get the assignment domain for this table
+      AssignmentDomain domain = this.getAssignmentDomain(tableName);
       // Get the assignment plan for the new regions
-      assignmentPlan = regionPlacement.getAssignmentPlan(newRegions);
+      assignmentPlan =
+        regionPlacement.getNewAssignmentPlan(newRegions, domain);
+    }
+
+    if (assignmentPlan == null) {
+      LOG.info("Generated the assignment plan for new table " + tableName);
+    } else {
+      LOG.info("NO assignment plan for new table " + tableName);
     }
 
     for(HRegionInfo newRegion : newRegions) {
@@ -1379,6 +1390,31 @@ public class HMaster extends Thread impl
     regionManager.metaScannerThread.triggerNow();
   }
 
+  /**
+   * Get the assignment domain for the table.
+   * Currently the domain would be generated by shuffling all the online
+   * region servers.
+   *
+   * It would be easy to extend for the multi-tenancy in the future.
+   * @param tableName
+   * @return the assignment domain for the table.
+   */
+  private AssignmentDomain getAssignmentDomain(String tableName) {
+    // Get all the online region servers
+    List<HServerAddress> onlineRSList =
+      this.serverManager.getOnlineRegionServerList();
+
+    // Shuffle the server list based on the tableName
+    Random random = new Random(tableName.hashCode());
+    Collections.shuffle(onlineRSList, random);
+
+    // Add the shuffled server list into the assignment domain
+    AssignmentDomain domain = new AssignmentDomain(this.conf);
+    domain.addServers(onlineRSList);
+
+    return domain;
+  }
+
   @Override
   public void deleteTable(final byte [] tableName) throws IOException {
     if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {

Copied: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RackManager.java (from r1310606, hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RackManager.java)
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RackManager.java?p2=hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RackManager.java&p1=hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RackManager.java&r1=1310606&r2=1310607&rev=1310607&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/RackManager.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RackManager.java Fri Apr  6 22:11:17 2012
@@ -1,4 +1,4 @@
-package org.apache.hadoop.hbase.util;
+package org.apache.hadoop.hbase.master;
 
 import java.util.Arrays;
 import java.util.List;
@@ -7,6 +7,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.IPv4AddressTruncationMapping;
@@ -40,9 +41,23 @@ public class RackManager {
    * @return the rack name of the server
    */
   public String getRack(HServerInfo info) {
+    if (info == null)
+      return HConstants.UNKNOWN_RACK;
+    return this.getRack(info.getServerAddress());
+  }
+
+  /**
+   * Get the name of the rack containing a server, according to the DNS to
+   * switch mapping.
+   * @param server the server for which to get the rack name
+   * @return the rack name of the server
+   */
+  public String getRack(HServerAddress server) {
+    if (server == null)
+      return HConstants.UNKNOWN_RACK;
+
     List<String> racks = switchMapping.resolve(Arrays.asList(
-        new String[]{info.getServerAddress().getInetSocketAddress()
-            .getAddress().getHostAddress()}));
+        new String[]{server.getBindAddress()}));
     if (racks != null && racks.size() > 0) {
       return racks.get(0);
     }

Added: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionAssignmentSnapshot.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionAssignmentSnapshot.java?rev=1310607&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionAssignmentSnapshot.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionAssignmentSnapshot.java Fri Apr  6 22:11:17 2012
@@ -0,0 +1,177 @@
+/**
+ * Copyright 2011 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.master;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.client.MetaScanner;
+import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Writables;
+
+public class RegionAssignmentSnapshot {
+  private static final Log LOG = LogFactory.getLog(RegionAssignmentSnapshot.class
+      .getName());
+
+  private Configuration conf;
+
+  /** the table name to region map */
+  private final Map<String, List<HRegionInfo>> tableToRegionMap;
+  /** the region to region server map */
+  private final Map<HRegionInfo, HServerAddress> regionToRegionServerMap;
+  /** the regionServer to region map */
+  private final Map<HServerAddress, List<HRegionInfo>> regionServerToRegionMap;
+  /** the existing assignment plan in the META region */
+  private final AssignmentPlan exsitingAssignmentPlan;
+  /** The rack view for the current region server */
+  private final AssignmentDomain globalAssignmentDomain;
+
+  public RegionAssignmentSnapshot(Configuration conf) {
+    this.conf = conf;
+    tableToRegionMap = new HashMap<String, List<HRegionInfo>>();
+    regionToRegionServerMap = new HashMap<HRegionInfo, HServerAddress>();
+    regionServerToRegionMap = new HashMap<HServerAddress, List<HRegionInfo>>();
+    exsitingAssignmentPlan = new AssignmentPlan();
+    globalAssignmentDomain = new AssignmentDomain(conf);
+  }
+
+  /**
+   * Initialize the region assignment snapshot by scanning the META table
+   * @throws IOException
+   */
+  public void initialize() throws IOException {
+    LOG.info("Start to scan the META for the current region assignment " +
+		"snappshot");
+
+    MetaScannerVisitor visitor = new MetaScannerVisitor() {
+      public boolean processRow(Result result) throws IOException {
+        try {
+          byte[] region = result.getValue(HConstants.CATALOG_FAMILY,
+              HConstants.REGIONINFO_QUALIFIER);
+          byte[] server = result.getValue(HConstants.CATALOG_FAMILY,
+              HConstants.SERVER_QUALIFIER);
+          // Process the region info
+          if (region == null) return true;
+          HRegionInfo regionInfo = Writables.getHRegionInfo(region);
+          if (regionInfo == null || regionInfo.isSplit()) {
+            return true;
+          }
+          addRegion(regionInfo);
+
+          // Process the region server
+          if (server == null) return true;
+          HServerAddress regionServer =
+            new HServerAddress(Bytes.toString(server));
+
+          // Add the current assignment to the snapshot
+          addAssignment(regionInfo, regionServer);
+          // Add the region server into the rack view
+          globalAssignmentDomain.addServer(regionServer);
+
+          // Process the assignment plan
+          byte[] favoredNodes = result.getValue(HConstants.CATALOG_FAMILY,
+              HConstants.FAVOREDNODES_QUALIFIER);
+          if (favoredNodes == null) return true;
+          // Add the favored nodes into assignment plan
+          List<HServerAddress> favoredServerList =
+            RegionPlacement.getFavoredNodesList(favoredNodes);
+          exsitingAssignmentPlan.updateAssignmentPlan(regionInfo,
+              favoredServerList);
+
+          return true;
+        } catch (RuntimeException e) {
+          LOG.error("Catche remote exception " + e.getMessage() +
+              " when processing" + result);
+          throw e;
+        }
+      }
+    };
+
+    // Scan .META. to pick up user regions
+    MetaScanner.metaScan(conf, visitor);
+    LOG.info("Finished to scan the META for the current region assignment" +
+      "snapshot");
+  }
+
+  private void addRegion(HRegionInfo regionInfo) {
+    if (regionInfo == null)
+      return;
+    // Process the table to region map
+    String tableName = regionInfo.getTableDesc().getNameAsString();
+    List<HRegionInfo> regionList = tableToRegionMap.get(tableName);
+    if (regionList == null) {
+      regionList = new ArrayList<HRegionInfo>();
+    }
+    // Add the current region info into the tableToRegionMap
+    regionList.add(regionInfo);
+    tableToRegionMap.put(tableName, regionList);
+  }
+
+  private void addAssignment(HRegionInfo regionInfo, HServerAddress server) {
+    if (server != null && regionInfo != null) {
+      // Process the region to region server map
+      regionToRegionServerMap.put(regionInfo, server);
+
+      // Process the region server to region map
+      List<HRegionInfo> regionList = regionServerToRegionMap.get(server);
+      if (regionList == null) {
+        regionList = new ArrayList<HRegionInfo>();
+      }
+      regionList.add(regionInfo);
+      regionServerToRegionMap.put(server, regionList);
+    }
+  }
+
+  public Map<String, List<HRegionInfo>> getTableToRegionMap() {
+    return tableToRegionMap;
+  }
+
+  public Map<HRegionInfo, HServerAddress> getRegionToRegionServerMap() {
+    return regionToRegionServerMap;
+  }
+
+  public Map<HServerAddress, List<HRegionInfo>> getRegionServerToRegionMap() {
+    return regionServerToRegionMap;
+  }
+
+  public AssignmentPlan getExistingAssignmentPlan() {
+    return this.exsitingAssignmentPlan;
+  }
+
+  public AssignmentDomain getGlobalAssignmentDomain() {
+    return this.globalAssignmentDomain;
+  }
+
+  public Set<String> getTableSet() {
+    return this.tableToRegionMap.keySet();
+  }
+}

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java?rev=1310607&r1=1310606&r2=1310607&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/RegionManager.java Fri Apr  6 22:11:17 2012
@@ -1076,9 +1076,10 @@ public class RegionManager {
     // 3.2 Put the favorite nodes into meta.
     if (favoriteNodeList != null) {
       String favoredNodes = RegionPlacement.getFavoredNodes(favoriteNodeList);
-      favoredNodes = favoredNodes.substring(0, favoredNodes.length() - 1);
       put.add(HConstants.CATALOG_FAMILY, HConstants.FAVOREDNODES_QUALIFIER,
           EnvironmentEdgeManager.currentTimeMillis(), favoredNodes.getBytes());
+      LOG.info("Create the region " + info.getRegionNameAsString() +
+          " with favored nodes " + favoredNodes);
     }
 
     server.put(metaRegionName, put);