This is an automated email from the ASF dual-hosted git repository. sammichen pushed a commit to branch HDDS-1564 in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git
commit facd4033f5cc7bcab01e36d8893c470a884b545a Author: Li Cheng <[email protected]> AuthorDate: Wed Jan 8 17:37:17 2020 +0100 HDDS-2756. Handle pipeline creation failure in different way when it exceeds pipeline limit Closes #401 --- .../hdds/scm/pipeline/PipelinePlacementPolicy.java | 30 ++++++++++++---------- .../hdds/scm/pipeline/SCMPipelineManager.java | 1 - 2 files changed, 17 insertions(+), 14 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 bc65d14..f4a13e1 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 @@ -114,10 +114,10 @@ public final class PipelinePlacementPolicy extends SCMCommonPlacementPolicy { boolean meet = (nodeManager.getPipelinesCount(datanodeDetails) - pipelineNumDeductable) < heavyNodeCriteria; if (!meet) { - LOG.info("Pipeline Placement: can't place more pipeline on heavy " + - "datanodeļ¼ " + datanodeDetails.getUuid().toString() + " Heaviness: " + - nodeManager.getPipelinesCount(datanodeDetails) + " limit: " + - heavyNodeCriteria); + LOG.debug("Pipeline Placement: can't place more pipeline on heavy " + + "datanodeļ¼ " + datanodeDetails.getUuid().toString() + + " Heaviness: " + nodeManager.getPipelinesCount(datanodeDetails) + + " limit: " + heavyNodeCriteria); } return meet; } @@ -144,17 +144,19 @@ public final class PipelinePlacementPolicy extends SCMCommonPlacementPolicy { int initialHealthyNodesCount = healthyNodes.size(); String msg; if (initialHealthyNodesCount == 0) { - msg = "No healthy node found to allocate pipeline."; + msg = "No healthy nodes found to allocate pipeline."; LOG.error(msg); throw new SCMException(msg, SCMException.ResultCodes .FAILED_TO_FIND_HEALTHY_NODES); } if (initialHealthyNodesCount < nodesRequired) { - msg = String.format("Not enough healthy nodes to allocate pipeline. %d " + LOG.warn("Not enough healthy nodes to allocate pipeline. %d " + " datanodes required. Found %d", nodesRequired, initialHealthyNodesCount); - LOG.error(msg); + msg = String.format("Pipeline creation failed due to no sufficient" + + " healthy datanodes. Required %d. Found %d.", + nodesRequired, initialHealthyNodesCount); throw new SCMException(msg, SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); } @@ -166,13 +168,15 @@ public final class PipelinePlacementPolicy extends SCMCommonPlacementPolicy { .collect(Collectors.toList()); if (healthyList.size() < nodesRequired) { - msg = String.format("Unable to find enough nodes that meet " + + LOG.debug("Unable to find enough nodes that meet " + "the criteria that cannot engage in more than %d pipelines." + " Nodes required: %d Found: %d, healthy nodes count in " + "NodeManager: %d.", heavyNodeCriteria, nodesRequired, healthyList.size(), initialHealthyNodesCount); - LOG.error(msg); + msg = String.format("Pipeline creation failed due to not enough" + + " healthy datanodes after filter. Required %d. Found %d", + nodesRequired, initialHealthyNodesCount); throw new SCMException(msg, SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); } @@ -229,8 +233,8 @@ public final class PipelinePlacementPolicy extends SCMCommonPlacementPolicy { // First choose an anchor nodes randomly DatanodeDetails anchor = chooseNode(healthyNodes); if (anchor == null) { - LOG.error("Pipeline Placement: Unable to find the first healthy nodes " + - "that meet the criteria. Required nodes: {}, Found nodes: {}", + LOG.warn("Unable to find healthy nodes." + + " Required nodes: {}, Found nodes: {}", nodesRequired, results.size()); throw new SCMException("Unable to find required number of nodes.", SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE); @@ -245,7 +249,7 @@ public final class PipelinePlacementPolicy extends SCMCommonPlacementPolicy { healthyNodes, exclude, nodeManager.getClusterNetworkTopologyMap(), anchor); if (nodeOnDifferentRack == null) { - LOG.error("Pipeline Placement: Unable to find nodes on different racks " + + LOG.warn("Pipeline Placement: Unable to find nodes on different racks " + " that meet the criteria. Required nodes: {}, Found nodes: {}", nodesRequired, results.size()); throw new SCMException("Unable to find required number of nodes.", @@ -269,7 +273,7 @@ public final class PipelinePlacementPolicy extends SCMCommonPlacementPolicy { } if (results.size() < nodesRequired) { - LOG.error("Pipeline Placement: Unable to find the required number of " + + LOG.warn("Unable to find the required number of " + "healthy nodes that meet the criteria. Required nodes: {}, " + "Found nodes: {}", nodesRequired, results.size()); throw new SCMException("Unable to find required number of nodes.", diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java index 11e9916..f924b41 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java @@ -187,7 +187,6 @@ public class SCMPipelineManager implements PipelineManager { return pipeline; } catch (IOException ex) { metrics.incNumPipelineCreationFailed(); - LOG.error("Pipeline creation failed.", ex); throw ex; } finally { lock.writeLock().unlock(); --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
