From 23f6c8bf2a1765635b0d097415513eed50bc1646 Mon Sep 17 00:00:00 2001 From: Li Cheng Date: Sun, 2 Feb 2020 01:05:32 +0800 Subject: [PATCH 1/3] HDDS-2923 Add fall-back protection for rack awareness in pipeline creation. --- .../scm/pipeline/PipelinePlacementPolicy.java | 64 ++++++++++++++----- .../pipeline/TestPipelinePlacementPolicy.java | 26 ++++++++ 2 files changed, 74 insertions(+), 16 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java index 9d78063a4dcd..d3dc9631f6ba 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java @@ -208,6 +208,29 @@ public List chooseDatanodes( } } + // Fall back logic for node pick up. + DatanodeDetails fallBackPickNodes( + List nodeSet, List excludedNodes) + throws SCMException{ + DatanodeDetails node; + if (excludedNodes == null || excludedNodes.isEmpty()) { + node = chooseNode(nodeSet); + } else { + List inputNodes = nodeSet.stream() + .filter(p -> !excludedNodes.contains(p)).collect(Collectors.toList()); + node = chooseNode(inputNodes); + } + + if (node == null) { + String msg = String.format("Unable to find fall back node in" + + " pipeline allocation. nodeSet size: {}", nodeSet.size()); + LOG.warn(msg); + throw new SCMException(msg, + SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); + } + return node; + } + /** * Get result set based on the pipeline placement algorithm which considers * network topology and rack awareness. @@ -220,6 +243,13 @@ public List chooseDatanodes( public List getResultSet( int nodesRequired, List healthyNodes) throws SCMException { + if (nodesRequired != HddsProtos.ReplicationFactor.THREE.getNumber()) { + throw new SCMException("Nodes required number is not supported: " + + nodesRequired, SCMException.ResultCodes.INVALID_CAPACITY); + } + + // Assume rack awareness is enabled. + boolean rackAwareness = true; List results = new ArrayList<>(nodesRequired); // Since nodes are widely distributed, the results should be selected // base on distance in topology, rack awareness and load balancing. @@ -227,10 +257,8 @@ public List getResultSet( // First choose an anchor nodes randomly DatanodeDetails anchor = chooseNode(healthyNodes); if (anchor == null) { - LOG.warn("Unable to find healthy node for anchor(first) node." + - " Required nodes: {}, Found nodes: {}", - nodesRequired, results.size()); - throw new SCMException("Unable to find required number of nodes.", + LOG.warn("Unable to find healthy node for anchor(first) node."); + throw new SCMException("Unable to find anchor node.", SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); } if (LOG.isDebugEnabled()) { @@ -241,29 +269,30 @@ public List getResultSet( exclude.add(anchor); // Choose the second node on different racks from anchor. - DatanodeDetails nodeOnDifferentRack = chooseNodeBasedOnRackAwareness( + DatanodeDetails nextNode = chooseNodeBasedOnRackAwareness( healthyNodes, exclude, nodeManager.getClusterNetworkTopologyMap(), anchor); - if (nodeOnDifferentRack == null) { - LOG.warn("Pipeline Placement: Unable to find 2nd node on different " + - "racks that meets the criteria. Required nodes: {}, Found nodes:" + - " {}", nodesRequired, results.size()); - throw new SCMException("Unable to find required number of nodes.", - SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); + if (nextNode == null) { + LOG.debug("Pipeline Placement: Unable to find 2nd node on different " + + "rack based on rack awareness."); + rackAwareness = false; + nextNode = fallBackPickNodes(healthyNodes, exclude); } if (LOG.isDebugEnabled()) { - LOG.debug("Second node chosen: {}", nodeOnDifferentRack); + LOG.debug("Second node chosen: {}", nextNode); } - results.add(nodeOnDifferentRack); - exclude.add(nodeOnDifferentRack); + results.add(nextNode); + exclude.add(nextNode); // Then choose nodes close to anchor based on network topology int nodesToFind = nodesRequired - results.size(); for (int x = 0; x < nodesToFind; x++) { // invoke the choose function defined in the derived classes. - DatanodeDetails pick = chooseNodeFromNetworkTopology( - nodeManager.getClusterNetworkTopologyMap(), anchor, exclude); + DatanodeDetails pick = rackAwareness + ? chooseNodeFromNetworkTopology( + nodeManager.getClusterNetworkTopologyMap(), anchor, exclude) + : fallBackPickNodes(healthyNodes, exclude); if (pick != null) { results.add(pick); exclude.add(pick); @@ -293,6 +322,9 @@ public List getResultSet( @Override public DatanodeDetails chooseNode( List healthyNodes) { + if (healthyNodes == null || healthyNodes.isEmpty()) { + return null; + } int firstNodeNdx = getRand().nextInt(healthyNodes.size()); int secondNodeNdx = getRand().nextInt(healthyNodes.size()); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java index b9aa9afb0518..e39956eedf87 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java @@ -83,10 +83,36 @@ public void testChooseNodeBasedOnRackAwareness() { DatanodeDetails nextNode = placementPolicy.chooseNodeBasedOnRackAwareness( healthyNodes, new ArrayList<>(PIPELINE_PLACEMENT_MAX_NODES_COUNT), topologyWithDifRacks, anchor); + Assert.assertNotNull(nextNode); Assert.assertFalse(anchor.getNetworkLocation().equals( nextNode.getNetworkLocation())); } + @Test + public void testFallBackPickNodes() { + List healthyNodes = overWriteLocationInNodes( + nodeManager.getNodes(HddsProtos.NodeState.HEALTHY)); + DatanodeDetails node; + try { + node = placementPolicy.fallBackPickNodes(healthyNodes, null); + Assert.assertNotNull(node); + } catch (SCMException e) { + Assert.fail("Should not reach here."); + } + + // when input nodeSet are all excluded. + List exclude = healthyNodes; + try { + node = placementPolicy.fallBackPickNodes(healthyNodes, exclude); + Assert.assertNull(node); + } catch (SCMException e) { + Assert.assertEquals(SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE, + e.getResult()); + } catch (Exception ex) { + Assert.fail("Should not reach here."); + } + } + private final static Node[] NODES = new NodeImpl[] { new NodeImpl("h1", "/r1", NetConstants.NODE_COST_DEFAULT), new NodeImpl("h2", "/r1", NetConstants.NODE_COST_DEFAULT), From 572195da277f70d9db62ec76a4516fd6a7692c6e Mon Sep 17 00:00:00 2001 From: Li Cheng Date: Thu, 6 Feb 2020 11:54:36 +0800 Subject: [PATCH 2/3] Add tests for pipeline placement policy when rack awareness is not enaled. --- .../scm/pipeline/PipelinePlacementPolicy.java | 25 ++++++++-------- .../pipeline/TestPipelinePlacementPolicy.java | 30 +++++++++++++++++++ 2 files changed, 43 insertions(+), 12 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java index d3dc9631f6ba..9032c3020be8 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java @@ -248,15 +248,18 @@ public List getResultSet( nodesRequired, SCMException.ResultCodes.INVALID_CAPACITY); } - // Assume rack awareness is enabled. - boolean rackAwareness = true; + // Assume rack awareness is not enabled. + boolean rackAwareness = false; List results = new ArrayList<>(nodesRequired); // Since nodes are widely distributed, the results should be selected // base on distance in topology, rack awareness and load balancing. List exclude = new ArrayList<>(); // First choose an anchor nodes randomly DatanodeDetails anchor = chooseNode(healthyNodes); - if (anchor == null) { + if (anchor != null) { + results.add(anchor); + exclude.add(anchor); + } else { LOG.warn("Unable to find healthy node for anchor(first) node."); throw new SCMException("Unable to find anchor node.", SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); @@ -265,30 +268,28 @@ public List getResultSet( LOG.debug("First node chosen: {}", anchor); } - results.add(anchor); - exclude.add(anchor); // Choose the second node on different racks from anchor. DatanodeDetails nextNode = chooseNodeBasedOnRackAwareness( healthyNodes, exclude, nodeManager.getClusterNetworkTopologyMap(), anchor); - if (nextNode == null) { + if (nextNode != null) { + // Rack awareness is detected. + rackAwareness = true; + results.add(nextNode); + exclude.add(nextNode); + } else { LOG.debug("Pipeline Placement: Unable to find 2nd node on different " + "rack based on rack awareness."); - rackAwareness = false; - nextNode = fallBackPickNodes(healthyNodes, exclude); } if (LOG.isDebugEnabled()) { LOG.debug("Second node chosen: {}", nextNode); } - results.add(nextNode); - exclude.add(nextNode); - // Then choose nodes close to anchor based on network topology int nodesToFind = nodesRequired - results.size(); for (int x = 0; x < nodesToFind; x++) { - // invoke the choose function defined in the derived classes. + // Pick remaining nodes based on the existence of rack awareness. DatanodeDetails pick = rackAwareness ? chooseNodeFromNetworkTopology( nodeManager.getClusterNetworkTopologyMap(), anchor, exclude) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java index e39956eedf87..daad80834c5a 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java @@ -113,6 +113,36 @@ public void testFallBackPickNodes() { } } + @Test + public void testRackAwarenessNotEnabledWithFallBack() throws SCMException{ + List healthyNodes = + nodeManager.getNodes(HddsProtos.NodeState.HEALTHY); + DatanodeDetails anchor = placementPolicy.chooseNode(healthyNodes); + DatanodeDetails randomNode = placementPolicy.chooseNode(healthyNodes); + // rack awareness is not enabled. + Assert.assertTrue(anchor.getNetworkLocation().equals( + randomNode.getNetworkLocation())); + + NetworkTopology topology = new NetworkTopologyImpl(new Configuration()); + DatanodeDetails nextNode = placementPolicy.chooseNodeBasedOnRackAwareness( + healthyNodes, new ArrayList<>(PIPELINE_PLACEMENT_MAX_NODES_COUNT), + topology, anchor); + // RackAwareness should not be able to choose any node. + Assert.assertNull(nextNode); + + // PlacementPolicy should still be able to pick a set of 3 nodes. + int numOfNodes = HddsProtos.ReplicationFactor.THREE.getNumber(); + List results = placementPolicy + .getResultSet(numOfNodes, healthyNodes); + + Assert.assertEquals(numOfNodes, results.size()); + // All nodes are on same rack. + Assert.assertEquals(results.get(0).getNetworkLocation(), + results.get(1).getNetworkLocation()); + Assert.assertEquals(results.get(0).getNetworkLocation(), + results.get(2).getNetworkLocation()); + } + private final static Node[] NODES = new NodeImpl[] { new NodeImpl("h1", "/r1", NetConstants.NODE_COST_DEFAULT), new NodeImpl("h2", "/r1", NetConstants.NODE_COST_DEFAULT), From 76aefb91aff2ee12e343bd784508a02941746736 Mon Sep 17 00:00:00 2001 From: Li Cheng Date: Sun, 9 Feb 2020 18:10:11 +0800 Subject: [PATCH 3/3] Modify debug message. --- .../hdds/scm/pipeline/PipelinePlacementPolicy.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java index 9032c3020be8..0f30449c9750 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java @@ -278,12 +278,14 @@ public List getResultSet( rackAwareness = true; results.add(nextNode); exclude.add(nextNode); + if (LOG.isDebugEnabled()) { + LOG.debug("Second node chosen: {}", nextNode); + } } else { - LOG.debug("Pipeline Placement: Unable to find 2nd node on different " + - "rack based on rack awareness."); - } - if (LOG.isDebugEnabled()) { - LOG.debug("Second node chosen: {}", nextNode); + if (LOG.isDebugEnabled()) { + LOG.debug("Pipeline Placement: Unable to find 2nd node on different " + + "rack based on rack awareness."); + } } // Then choose nodes close to anchor based on network topology