浏览代码

HDFS-6268. Better sorting in NetworkTopology#pseudoSortByDistance when no local node is found. (wang)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1599734 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 年之前
父节点
当前提交
02fcb6b6ba

+ 91 - 52
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -19,8 +19,10 @@ package org.apache.hadoop.net;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Random;
+import java.util.TreeMap;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -32,6 +34,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
 /** The class represents a cluster of computer with a tree hierarchical
  * network topology.
  * For example, a cluster may be consists of many data centers filled 
@@ -667,7 +672,23 @@ public class NetworkTopology {
     return node1.getParent()==node2.getParent();
   }
 
-  final protected static Random r = new Random();
+  private static final ThreadLocal<Random> r = new ThreadLocal<Random>();
+
+  /**
+   * Getter for thread-local Random, which provides better performance than
+   * a shared Random (even though Random is thread-safe).
+   *
+   * @return Thread-local Random.
+   */
+  protected Random getRandom() {
+    Random rand = r.get();
+    if (rand == null) {
+      rand = new Random();
+      r.set(rand);
+    }
+    return rand;
+  }
+
   /** randomly choose one node from <i>scope</i>
    * if scope starts with ~, choose one from the all nodes except for the
    * ones in <i>scope</i>; otherwise, choose one from <i>scope</i>
@@ -717,7 +738,7 @@ public class NetworkTopology {
           "Failed to find datanode (scope=\"" + String.valueOf(scope) +
           "\" excludedScope=\"" + String.valueOf(excludedScope) + "\").");
     }
-    int leaveIndex = r.nextInt(numOfDatanodes);
+    int leaveIndex = getRandom().nextInt(numOfDatanodes);
     return innerNode.getLeaf(leaveIndex, node);
   }
 
@@ -824,61 +845,79 @@ public class NetworkTopology {
     return networkLocation.substring(index);
   }
 
-  /** swap two array items */
-  static protected void swap(Node[] nodes, int i, int j) {
-    Node tempNode;
-    tempNode = nodes[j];
-    nodes[j] = nodes[i];
-    nodes[i] = tempNode;
-  }
-  
-  /** Sort nodes array by their distances to <i>reader</i>
-   * It linearly scans the array, if a local node is found, swap it with
-   * the first element of the array.
-   * If a local rack node is found, swap it with the first element following
-   * the local node.
-   * If neither local node or local rack node is found, put a random replica
-   * location at position 0.
-   * It leaves the rest nodes untouched.
-   * @param reader the node that wishes to read a block from one of the nodes
-   * @param nodes the list of nodes containing data for the reader
+  /**
+   * Returns an integer weight which specifies how far away {node} is away from
+   * {reader}. A lower value signifies that a node is closer.
+   * 
+   * @param reader Node where data will be read
+   * @param node Replica of data
+   * @return weight
    */
-  public void pseudoSortByDistance( Node reader, Node[] nodes ) {
-    int tempIndex = 0;
-    int localRackNode = -1;
-    if (reader != null ) {
-      //scan the array to find the local node & local rack node
-      for(int i=0; i<nodes.length; i++) {
-        if(tempIndex == 0 && reader == nodes[i]) { //local node
-          //swap the local node and the node at position 0
-          if( i != 0 ) {
-            swap(nodes, tempIndex, i);
-          }
-          tempIndex=1;
-          if(localRackNode != -1 ) {
-            if(localRackNode == 0) {
-              localRackNode = i;
-            }
-            break;
-          }
-        } else if(localRackNode == -1 && isOnSameRack(reader, nodes[i])) {
-          //local rack
-          localRackNode = i;
-          if(tempIndex != 0 ) break;
-        }
+  protected int getWeight(Node reader, Node node) {
+    // 0 is local, 1 is same rack, 2 is off rack
+    // Start off by initializing to off rack
+    int weight = 2;
+    if (reader != null) {
+      if (reader == node) {
+        weight = 0;
+      } else if (isOnSameRack(reader, node)) {
+        weight = 1;
       }
+    }
+    return weight;
+  }
 
-      // swap the local rack node and the node at position tempIndex
-      if(localRackNode != -1 && localRackNode != tempIndex ) {
-        swap(nodes, tempIndex, localRackNode);
-        tempIndex++;
+  /**
+   * Sort nodes array by network distance to <i>reader</i>.
+   * <p/>
+   * In a three-level topology, a node can be either local, on the same rack, or
+   * on a different rack from the reader. Sorting the nodes based on network
+   * distance from the reader reduces network traffic and improves performance.
+   * <p/>
+   * As an additional twist, we also randomize the nodes at each network
+   * distance using the provided random seed. This helps with load balancing
+   * when there is data skew.
+   * 
+   * @param reader Node where data will be read
+   * @param nodes Available replicas with the requested data
+   * @param seed Used to seed the pseudo-random generator that randomizes the
+   *          set of nodes at each network distance.
+   */
+  public void sortByDistance(Node reader, Node[] nodes, long seed) {
+    /** Sort weights for the nodes array */
+    int[] weights = new int[nodes.length];
+    for (int i=0; i<nodes.length; i++) {
+      weights[i] = getWeight(reader, nodes[i]);
+    }
+    // Add weight/node pairs to a TreeMap to sort
+    TreeMap<Integer, List<Node>> tree = new TreeMap<Integer, List<Node>>();
+    for (int i=0; i<nodes.length; i++) {
+      int weight = weights[i];
+      Node node = nodes[i];
+      List<Node> list = tree.get(weight);
+      if (list == null) {
+        list = Lists.newArrayListWithExpectedSize(1);
+        tree.put(weight, list);
       }
+      list.add(node);
     }
-    
-    // put a random node at position 0 if it is not a local/local-rack node
-    if(tempIndex == 0 && localRackNode == -1 && nodes.length != 0) {
-      swap(nodes, 0, r.nextInt(nodes.length));
+
+    // Seed is normally the block id
+    // This means we use the same pseudo-random order for each block, for
+    // potentially better page cache usage.
+    Random rand = getRandom();
+    rand.setSeed(seed);
+    int idx = 0;
+    for (List<Node> list: tree.values()) {
+      if (list != null) {
+        Collections.shuffle(list, rand);
+        for (Node n: list) {
+          nodes[idx] = n;
+          idx++;
+        }
+      }
     }
+    Preconditions.checkState(idx == nodes.length,
+        "Sorted the wrong number of nodes!");
   }
-  
 }

+ 33 - 72
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java

@@ -248,25 +248,41 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
     }
   }
 
-  /** Sort nodes array by their distances to <i>reader</i>
-   * It linearly scans the array, if a local node is found, swap it with
-   * the first element of the array.
-   * If a local node group node is found, swap it with the first element 
-   * following the local node.
-   * If a local rack node is found, swap it with the first element following
-   * the local node group node.
-   * If neither local node, node group node or local rack node is found, put a 
-   * random replica location at position 0.
-   * It leaves the rest nodes untouched.
-   * @param reader the node that wishes to read a block from one of the nodes
-   * @param nodes the list of nodes containing data for the reader
-   */
   @Override
-  public void pseudoSortByDistance( Node reader, Node[] nodes ) {
+  protected int getWeight(Node reader, Node node) {
+    // 0 is local, 1 is same node group, 2 is same rack, 3 is off rack
+    // Start off by initializing to off rack
+    int weight = 3;
+    if (reader != null) {
+      if (reader == node) {
+        weight = 0;
+      } else if (isOnSameNodeGroup(reader, node)) {
+        weight = 1;
+      } else if (isOnSameRack(reader, node)) {
+        weight = 2;
+      }
+    }
+    return weight;
+  }
 
+  /**
+   * Sort nodes array by their distances to <i>reader</i>.
+   * <p/>
+   * This is the same as
+   * {@link NetworkTopology#sortByDistance(Node, Node[], long)} except with a
+   * four-level network topology which contains the additional network distance
+   * of a "node group" which is between local and same rack.
+   * 
+   * @param reader Node where data will be read
+   * @param nodes Available replicas with the requested data
+   * @param seed Used to seed the pseudo-random generator that randomizes the
+   *          set of nodes at each network distance.
+   */
+  @Override
+  public void sortByDistance( Node reader, Node[] nodes, long seed) {
+    // If reader is not a datanode (not in NetworkTopology tree), we need to
+    // replace this reader with a sibling leaf node in tree.
     if (reader != null && !this.contains(reader)) {
-      // if reader is not a datanode (not in NetworkTopology tree), we will 
-      // replace this reader with a sibling leaf node in tree.
       Node nodeGroup = getNode(reader.getNetworkLocation());
       if (nodeGroup != null && nodeGroup instanceof InnerNode) {
         InnerNode parentNode = (InnerNode) nodeGroup;
@@ -276,62 +292,7 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
         return;
       }
     }
-    int tempIndex = 0;
-    int localRackNode = -1;
-    int localNodeGroupNode = -1;
-    if (reader != null) {  
-      //scan the array to find the local node & local rack node
-      for (int i = 0; i < nodes.length; i++) {
-        if (tempIndex == 0 && reader == nodes[i]) { //local node
-          //swap the local node and the node at position 0
-          if (i != 0) {
-            swap(nodes, tempIndex, i);
-          }
-          tempIndex=1;
-
-          if (localRackNode != -1 && (localNodeGroupNode !=-1)) {
-            if (localRackNode == 0) {
-              localRackNode = i;
-            }
-            if (localNodeGroupNode == 0) {
-              localNodeGroupNode = i;
-            }
-            break;
-          }
-        } else if (localNodeGroupNode == -1 && isOnSameNodeGroup(reader, 
-            nodes[i])) {
-          //local node group
-          localNodeGroupNode = i;
-          // node local and rack local are already found
-          if(tempIndex != 0 && localRackNode != -1) break;
-        } else if (localRackNode == -1 && isOnSameRack(reader, nodes[i])) {
-          localRackNode = i;
-          if (tempIndex != 0 && localNodeGroupNode != -1) break;
-        }
-      }
-
-      // swap the local nodegroup node and the node at position tempIndex
-      if(localNodeGroupNode != -1 && localNodeGroupNode != tempIndex) {
-        swap(nodes, tempIndex, localNodeGroupNode);
-        if (localRackNode == tempIndex) {
-          localRackNode = localNodeGroupNode;
-        }
-        tempIndex++;
-      }
-
-      // swap the local rack node and the node at position tempIndex
-      if(localRackNode != -1 && localRackNode != tempIndex) {
-        swap(nodes, tempIndex, localRackNode);
-        tempIndex++;
-      }
-    }
-
-    // put a random node at position 0 if there is not a local/local-nodegroup/
-    // local-rack node
-    if (tempIndex == 0 && localNodeGroupNode == -1 && localRackNode == -1
-        && nodes.length != 0) {
-      swap(nodes, 0, r.nextInt(nodes.length));
-    }
+    super.sortByDistance(reader, nodes, seed);
   }
 
   /** InnerNodeWithNodeGroup represents a switch/router of a data center, rack

+ 5 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java

@@ -96,7 +96,7 @@ public class TestNetworkTopologyWithNodeGroup {
   }
 
   @Test
-  public void testPseudoSortByDistance() throws Exception {
+  public void testSortByDistance() throws Exception {
     NodeBase[] testNodes = new NodeBase[4];
 
     // array contains both local node, local node group & local rack node
@@ -104,7 +104,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[2];
     testNodes[2] = dataNodes[3];
     testNodes[3] = dataNodes[0];
-    cluster.pseudoSortByDistance(dataNodes[0], testNodes );
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
     assertTrue(testNodes[2] == dataNodes[2]);
@@ -115,7 +115,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[4];
     testNodes[2] = dataNodes[1];
     testNodes[3] = dataNodes[0];
-    cluster.pseudoSortByDistance(dataNodes[0], testNodes );
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
 
@@ -124,7 +124,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[3];
     testNodes[2] = dataNodes[2];
     testNodes[3] = dataNodes[0];
-    cluster.pseudoSortByDistance(dataNodes[0], testNodes );
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[2]);
 
@@ -133,7 +133,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[7];
     testNodes[2] = dataNodes[2];
     testNodes[3] = dataNodes[0];
-    cluster.pseudoSortByDistance(computeNode, testNodes );
+    cluster.sortByDistance(computeNode, testNodes, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[2]);
   }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -467,6 +467,9 @@ Release 2.5.0 - UNRELEASED
     HDFS-6109 let sync_file_range() system call run in background
     (Liang Xie via stack)
 
+    HDFS-6268. Better sorting in NetworkTopology#pseudoSortByDistance when
+    no local node is found. (wang)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -351,7 +351,8 @@ public class DatanodeManager {
         DFSUtil.DECOM_COMPARATOR;
         
     for (LocatedBlock b : locatedblocks) {
-      networktopology.pseudoSortByDistance(client, b.getLocations());
+      networktopology.sortByDistance(client, b.getLocations(), b
+          .getBlock().getBlockId());
       // Move decommissioned/stale datanodes to the bottom
       Arrays.sort(b.getLocations(), comparator);
     }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1633,9 +1633,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       blockManager.getDatanodeManager().sortLocatedBlocks(
           clientMachine, blocks.getLocatedBlocks());
       
+      // lastBlock is not part of getLocatedBlocks(), might need to sort it too
       LocatedBlock lastBlock = blocks.getLastLocatedBlock();
       if (lastBlock != null) {
-        ArrayList<LocatedBlock> lastBlockList = new ArrayList<LocatedBlock>();
+        ArrayList<LocatedBlock> lastBlockList =
+            Lists.newArrayListWithCapacity(1);
         lastBlockList.add(lastBlock);
         blockManager.getDatanodeManager().sortLocatedBlocks(
                               clientMachine, lastBlockList);

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java

@@ -167,6 +167,9 @@ public class TestGetBlocks {
       if (stm != null) {
         stm.close();
       }
+      if (client != null) {
+        client.close();
+      }
       cluster.shutdown();
     }
   }

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java

@@ -143,10 +143,10 @@ public class TestSnapshotFileLength {
 
     // Make sure we can read the entire file via its non-snapshot path.
     fileStatus = hdfs.getFileStatus(file1);
-    assertEquals(fileStatus.getLen(), BLOCKSIZE * 2);
+    assertEquals("Unexpected file length", BLOCKSIZE * 2, fileStatus.getLen());
     fis = hdfs.open(file1);
     bytesRead = fis.read(buffer, 0, buffer.length);
-    assertEquals(bytesRead, BLOCKSIZE * 2);
+    assertEquals("Unexpected # bytes read", BLOCKSIZE * 2, bytesRead);
     fis.close();
 
     Path file1snap1 =
@@ -156,21 +156,23 @@ public class TestSnapshotFileLength {
     assertEquals(fileStatus.getLen(), BLOCKSIZE);
     // Make sure we can only read up to the snapshot length.
     bytesRead = fis.read(buffer, 0, buffer.length);
-    assertEquals(bytesRead, BLOCKSIZE);
+    assertEquals("Unexpected # bytes read", BLOCKSIZE, bytesRead);
     fis.close();
 
-    PrintStream psBackup = System.out;
+    PrintStream outBackup = System.out;
+    PrintStream errBackup = System.err;
     ByteArrayOutputStream bao = new ByteArrayOutputStream();
     System.setOut(new PrintStream(bao));
     System.setErr(new PrintStream(bao));
     // Make sure we can cat the file upto to snapshot length
     FsShell shell = new FsShell();
-    try{
+    try {
       ToolRunner.run(conf, shell, new String[] { "-cat",
       "/TestSnapshotFileLength/sub1/.snapshot/snapshot1/file1" });
-      assertEquals(bao.size(), BLOCKSIZE);
-    }finally{
-      System.setOut(psBackup);
+      assertEquals("Unexpected # bytes from -cat", BLOCKSIZE, bao.size());
+    } finally {
+      System.setOut(outBackup);
+      System.setErr(errBackup);
     }
   }
 }

+ 38 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java

@@ -54,7 +54,8 @@ public class TestNetworkTopology {
         DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2"),
         DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2"),
         DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d2/r3"),
-        DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3")
+        DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3"),
+        DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3")
     };
     for (int i = 0; i < dataNodes.length; i++) {
       cluster.add(dataNodes[i]);
@@ -117,14 +118,14 @@ public class TestNetworkTopology {
   }
 
   @Test
-  public void testPseudoSortByDistance() throws Exception {
+  public void testSortByDistance() throws Exception {
     DatanodeDescriptor[] testNodes = new DatanodeDescriptor[3];
     
     // array contains both local node & local rack node
     testNodes[0] = dataNodes[1];
     testNodes[1] = dataNodes[2];
     testNodes[2] = dataNodes[0];
-    cluster.pseudoSortByDistance(dataNodes[0], testNodes );
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
     assertTrue(testNodes[2] == dataNodes[2]);
@@ -133,7 +134,7 @@ public class TestNetworkTopology {
     testNodes[0] = dataNodes[1];
     testNodes[1] = dataNodes[3];
     testNodes[2] = dataNodes[0];
-    cluster.pseudoSortByDistance(dataNodes[0], testNodes );
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
     assertTrue(testNodes[2] == dataNodes[3]);
@@ -142,21 +143,50 @@ public class TestNetworkTopology {
     testNodes[0] = dataNodes[5];
     testNodes[1] = dataNodes[3];
     testNodes[2] = dataNodes[1];
-    cluster.pseudoSortByDistance(dataNodes[0], testNodes );
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[1]);
     assertTrue(testNodes[1] == dataNodes[3]);
     assertTrue(testNodes[2] == dataNodes[5]);
-    
+
     // array contains local rack node which happens to be in position 0
     testNodes[0] = dataNodes[1];
     testNodes[1] = dataNodes[5];
     testNodes[2] = dataNodes[3];
-    cluster.pseudoSortByDistance(dataNodes[0], testNodes );
-    // peudoSortByDistance does not take the "data center" layer into consideration 
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    assertTrue(testNodes[0] == dataNodes[1]);
+    assertTrue(testNodes[1] == dataNodes[3]);
+    assertTrue(testNodes[2] == dataNodes[5]);
+
+    // Same as previous, but with a different random seed to test randomization
+    testNodes[0] = dataNodes[1];
+    testNodes[1] = dataNodes[5];
+    testNodes[2] = dataNodes[3];
+    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEAD);
+    // sortByDistance does not take the "data center" layer into consideration
     // and it doesn't sort by getDistance, so 1, 5, 3 is also valid here
     assertTrue(testNodes[0] == dataNodes[1]);
     assertTrue(testNodes[1] == dataNodes[5]);
     assertTrue(testNodes[2] == dataNodes[3]);
+
+    // Array is just local rack nodes
+    // Expect a random first node depending on the seed (normally the block ID).
+    DatanodeDescriptor first = null;
+    boolean foundRandom = false;
+    for (int i=5; i<=7; i++) {
+      testNodes[0] = dataNodes[5];
+      testNodes[1] = dataNodes[6];
+      testNodes[2] = dataNodes[7];
+      cluster.sortByDistance(dataNodes[i], testNodes, 0xBEADED+i);
+      if (first == null) {
+        first = testNodes[0];
+      } else {
+        if (first != testNodes[0]) {
+          foundRandom = true;
+          break;
+        }
+      }
+    }
+    assertTrue("Expected to find a different first location", foundRandom);
   }
   
   @Test