You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2017/08/23 19:49:31 UTC

hadoop git commit: HDFS-11514. DFSTopologyNodeImpl#chooseRandom optimizations. Contributed by Chen Liang.

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 6e8c69665 -> 8b2d5a4fa


HDFS-11514. DFSTopologyNodeImpl#chooseRandom optimizations. Contributed by Chen Liang.


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

Branch: refs/heads/branch-2
Commit: 8b2d5a4faa2b683d8049aa27bf8a5d82ee9e8683
Parents: 6e8c696
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Aug 23 12:48:59 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Wed Aug 23 12:49:13 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/net/DFSTopologyNodeImpl.java    | 125 +++++++++++--------
 .../hadoop/hdfs/net/TestDFSNetworkTopology.java |  64 +++++++++-
 2 files changed, 139 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b2d5a4f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DFSTopologyNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DFSTopologyNodeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DFSTopologyNodeImpl.java
index c00978d..f06f250 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DFSTopologyNodeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DFSTopologyNodeImpl.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.net.Node;
 import java.util.EnumMap;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.Map;
 
 /**
  * The HDFS-specific representation of a network topology inner node. The
@@ -76,32 +75,58 @@ public class DFSTopologyNodeImpl extends InnerNodeImpl {
   private final HashMap
       <String, EnumMap<StorageType, Integer>> childrenStorageInfo;
 
+  /**
+   * This map stores storage type counts of the subtree. We can always get this
+   * info by iterate over the childrenStorageInfo variable. But for optimization
+   * purpose, we store this info directly to avoid the iteration.
+   */
+  private final EnumMap<StorageType, Integer> storageTypeCounts;
+
   DFSTopologyNodeImpl(String path) {
     super(path);
     childrenStorageInfo = new HashMap<>();
+    storageTypeCounts = new EnumMap<>(StorageType.class);
   }
 
   DFSTopologyNodeImpl(
       String name, String location, InnerNode parent, int level) {
     super(name, location, parent, level);
     childrenStorageInfo = new HashMap<>();
+    storageTypeCounts = new EnumMap<>(StorageType.class);
   }
 
   public int getSubtreeStorageCount(StorageType type) {
-    int res = 0;
-    for (Map.Entry<String, EnumMap<StorageType, Integer>> entry :
-        childrenStorageInfo.entrySet()) {
-      if (entry.getValue().containsKey(type)) {
-        res += entry.getValue().get(type);
-      }
+    if (storageTypeCounts.containsKey(type)) {
+      return storageTypeCounts.get(type);
+    } else {
+      return 0;
     }
-    return res;
   }
 
   int getNumOfChildren() {
     return children.size();
   }
 
+  private void incStorageTypeCount(StorageType type) {
+    // no locking because the caller is synchronized already
+    if (storageTypeCounts.containsKey(type)) {
+      storageTypeCounts.put(type, storageTypeCounts.get(type)+1);
+    } else {
+      storageTypeCounts.put(type, 1);
+    }
+  }
+
+  private void decStorageTypeCount(StorageType type) {
+    // no locking because the caller is synchronized already
+    int current = storageTypeCounts.get(type);
+    current -= 1;
+    if (current == 0) {
+      storageTypeCounts.remove(type);
+    } else {
+      storageTypeCounts.put(type, current);
+    }
+  }
+
   @Override
   public boolean add(Node n) {
     if (!isAncestor(n)) {
@@ -130,15 +155,14 @@ public class DFSTopologyNodeImpl extends InnerNodeImpl {
       }
       children.add(n);
       numOfLeaves++;
-      synchronized (childrenStorageInfo) {
-        if (!childrenStorageInfo.containsKey(dnDescriptor.getName())) {
-          childrenStorageInfo.put(
-              dnDescriptor.getName(),
-              new EnumMap<StorageType, Integer>(StorageType.class));
-        }
-        for (StorageType st : dnDescriptor.getStorageTypes()) {
-          childrenStorageInfo.get(dnDescriptor.getName()).put(st, 1);
-        }
+      if (!childrenStorageInfo.containsKey(dnDescriptor.getName())) {
+        childrenStorageInfo.put(
+            dnDescriptor.getName(),
+            new EnumMap<StorageType, Integer>(StorageType.class));
+      }
+      for (StorageType st : dnDescriptor.getStorageTypes()) {
+        childrenStorageInfo.get(dnDescriptor.getName()).put(st, 1);
+        incStorageTypeCount(st);
       }
       return true;
     } else {
@@ -154,26 +178,27 @@ public class DFSTopologyNodeImpl extends InnerNodeImpl {
       // add n to the subtree of the next ancestor node
       if (parentNode.add(n)) {
         numOfLeaves++;
-        synchronized (childrenStorageInfo) {
-          if (!childrenStorageInfo.containsKey(parentNode.getName())) {
-            childrenStorageInfo.put(
-                parentNode.getName(),
-                new EnumMap<StorageType, Integer>(StorageType.class));
-            for (StorageType st : dnDescriptor.getStorageTypes()) {
-              childrenStorageInfo.get(parentNode.getName()).put(st, 1);
-            }
-          } else {
-            EnumMap<StorageType, Integer> currentCount =
-                childrenStorageInfo.get(parentNode.getName());
-            for (StorageType st : dnDescriptor.getStorageTypes()) {
-              if (currentCount.containsKey(st)) {
-                currentCount.put(st, currentCount.get(st) + 1);
-              } else {
-                currentCount.put(st, 1);
-              }
+        if (!childrenStorageInfo.containsKey(parentNode.getName())) {
+          childrenStorageInfo.put(
+              parentNode.getName(),
+              new EnumMap<StorageType, Integer>(StorageType.class));
+          for (StorageType st : dnDescriptor.getStorageTypes()) {
+            childrenStorageInfo.get(parentNode.getName()).put(st, 1);
+          }
+        } else {
+          EnumMap<StorageType, Integer> currentCount =
+              childrenStorageInfo.get(parentNode.getName());
+          for (StorageType st : dnDescriptor.getStorageTypes()) {
+            if (currentCount.containsKey(st)) {
+              currentCount.put(st, currentCount.get(st) + 1);
+            } else {
+              currentCount.put(st, 1);
             }
           }
         }
+        for (StorageType st : dnDescriptor.getStorageTypes()) {
+          incStorageTypeCount(st);
+        }
         return true;
       } else {
         return false;
@@ -222,8 +247,9 @@ public class DFSTopologyNodeImpl extends InnerNodeImpl {
           if (children.get(i).getName().equals(n.getName())) {
             children.remove(i);
             childrenMap.remove(n.getName());
-            synchronized (childrenStorageInfo) {
-              childrenStorageInfo.remove(dnDescriptor.getName());
+            childrenStorageInfo.remove(dnDescriptor.getName());
+            for (StorageType st : dnDescriptor.getStorageTypes()) {
+              decStorageTypeCount(st);
             }
             numOfLeaves--;
             n.setParent(null);
@@ -244,20 +270,21 @@ public class DFSTopologyNodeImpl extends InnerNodeImpl {
       boolean isRemoved = parentNode.remove(n);
       if (isRemoved) {
         // if the parent node has no children, remove the parent node too
-        synchronized (childrenStorageInfo) {
-          EnumMap<StorageType, Integer> currentCount =
-              childrenStorageInfo.get(parentNode.getName());
-          EnumSet<StorageType> toRemove = EnumSet.noneOf(StorageType.class);
-          for (StorageType st : dnDescriptor.getStorageTypes()) {
-            int newCount = currentCount.get(st) - 1;
-            if (newCount == 0) {
-              toRemove.add(st);
-            }
-            currentCount.put(st, newCount);
-          }
-          for (StorageType st : toRemove) {
-            currentCount.remove(st);
+        EnumMap<StorageType, Integer> currentCount =
+            childrenStorageInfo.get(parentNode.getName());
+        EnumSet<StorageType> toRemove = EnumSet.noneOf(StorageType.class);
+        for (StorageType st : dnDescriptor.getStorageTypes()) {
+          int newCount = currentCount.get(st) - 1;
+          if (newCount == 0) {
+            toRemove.add(st);
           }
+          currentCount.put(st, newCount);
+        }
+        for (StorageType st : toRemove) {
+          currentCount.remove(st);
+        }
+        for (StorageType st : dnDescriptor.getStorageTypes()) {
+          decStorageTypeCount(st);
         }
         if (parentNode.getNumOfChildren() == 0) {
           for(int i=0; i < children.size(); i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b2d5a4f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java
index 32ecf886..30ef2ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java
@@ -229,7 +229,6 @@ public class TestDFSNetworkTopology {
     assertEquals(1, (int)l1info.get("d2").get(StorageType.DISK));
     assertEquals(2, (int)l1info.get("d3").get(StorageType.SSD));
 
-
     for (int i = 0; i<4; i++) {
       CLUSTER.remove(newDD[i]);
     }
@@ -446,4 +445,67 @@ public class TestDFSNetworkTopology {
         "/l100/d100/r100", null, null, StorageType.DISK);
     assertNull(n);
   }
+
+  /**
+   * Tests getting subtree storage counts, and see whether it is correct when
+   * we update subtree.
+   * @throws Exception
+   */
+  @Test
+  public void testGetSubtreeStorageCount() throws Exception {
+    // add and remove a node to rack /l2/d3/r1. So all the inner nodes /l2,
+    // /l2/d3 and /l2/d3/r1 should be affected. /l2/d3/r3 should still be the
+    // same, only checked as a reference
+    Node l2 = CLUSTER.getNode("/l2");
+    Node l2d3 = CLUSTER.getNode("/l2/d3");
+    Node l2d3r1 = CLUSTER.getNode("/l2/d3/r1");
+    Node l2d3r3 = CLUSTER.getNode("/l2/d3/r3");
+
+    assertTrue(l2 instanceof DFSTopologyNodeImpl);
+    assertTrue(l2d3 instanceof DFSTopologyNodeImpl);
+    assertTrue(l2d3r1 instanceof DFSTopologyNodeImpl);
+    assertTrue(l2d3r3 instanceof DFSTopologyNodeImpl);
+
+    DFSTopologyNodeImpl innerl2 = (DFSTopologyNodeImpl)l2;
+    DFSTopologyNodeImpl innerl2d3 = (DFSTopologyNodeImpl)l2d3;
+    DFSTopologyNodeImpl innerl2d3r1 = (DFSTopologyNodeImpl)l2d3r1;
+    DFSTopologyNodeImpl innerl2d3r3 = (DFSTopologyNodeImpl)l2d3r3;
+
+    assertEquals(4,
+        innerl2.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(2,
+        innerl2d3.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(1,
+        innerl2d3r1.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(1,
+        innerl2d3r3.getSubtreeStorageCount(StorageType.DISK));
+
+    DatanodeStorageInfo storageInfo =
+        DFSTestUtil.createDatanodeStorageInfo("StorageID",
+            "1.2.3.4", "/l2/d3/r1", "newhost");
+    DatanodeDescriptor newNode = storageInfo.getDatanodeDescriptor();
+    CLUSTER.add(newNode);
+
+    // after adding a storage to /l2/d3/r1, ancestor inner node should have
+    // DISK count incremented by 1.
+    assertEquals(5,
+        innerl2.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(3,
+        innerl2d3.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(2,
+        innerl2d3r1.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(1,
+        innerl2d3r3.getSubtreeStorageCount(StorageType.DISK));
+
+    CLUSTER.remove(newNode);
+
+    assertEquals(4,
+        innerl2.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(2,
+        innerl2d3.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(1,
+        innerl2d3r1.getSubtreeStorageCount(StorageType.DISK));
+    assertEquals(1,
+        innerl2d3r3.getSubtreeStorageCount(StorageType.DISK));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org