Browse Source

HDDS-1787. NPE thrown while trying to find DN closest to client. Contributed by Sammi Chen. (#1094)

Sammi Chen 5 năm trước cách đây
mục cha
commit
493b0b5760

+ 15 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.protocol.proto
         .StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.net.InnerNode;
 import org.apache.hadoop.hdds.scm.net.NetConstants;
 import org.apache.hadoop.hdds.scm.net.NetworkTopology;
 import org.apache.hadoop.hdds.scm.net.Node;
@@ -566,9 +567,20 @@ public class SCMNodeManager implements NodeManager {
       node = clusterMap.getNode(location + NetConstants.PATH_SEPARATOR_STR +
           address);
     }
-    LOG.debug("Get node for {} return {}", address, (node == null ?
-        "not found" : node.getNetworkFullPath()));
-    return node == null ? null : (DatanodeDetails)node;
+
+    if (node != null) {
+      if (node instanceof InnerNode) {
+        LOG.warn("Get node for {} return {}, it's an inner node, " +
+            "not a datanode", address, node.getNetworkFullPath());
+      } else {
+        LOG.debug("Get node for {} return {}", address,
+            node.getNetworkFullPath());
+        return (DatanodeDetails)node;
+      }
+    } else {
+      LOG.warn("Cannot find node for {}", address);
+    }
+    return null;
   }
 
   private String nodeResolve(String hostname) {

+ 6 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java

@@ -290,7 +290,12 @@ public class SCMBlockProtocolServer implements
       NodeManager nodeManager = scm.getScmNodeManager();
       Node client = nodeManager.getNode(clientMachine);
       List<Node> nodeList = new ArrayList();
-      nodes.stream().forEach(path -> nodeList.add(nodeManager.getNode(path)));
+      nodes.stream().forEach(path -> {
+        DatanodeDetails node = nodeManager.getNode(path);
+        if (node != null) {
+          nodeList.add(nodeManager.getNode(path));
+        }
+      });
       List<? extends Node> sortedNodeList = scm.getClusterMap()
           .sortByDistanceCost(client, nodeList, nodes.size());
       List<DatanodeDetails> ret = new ArrayList<>();

+ 153 - 0
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMBlockProtocolServer.java

@@ -0,0 +1,153 @@
+/*
+ * 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.hdds.scm.server;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos;
+import org.apache.hadoop.hdds.scm.TestUtils;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.ozone.protocolPB
+    .ScmBlockLocationProtocolServerSideTranslatorPB;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test class for @{@link SCMBlockProtocolServer}.
+ * */
+public class TestSCMBlockProtocolServer {
+  private OzoneConfiguration config;
+  private SCMBlockProtocolServer server;
+  private StorageContainerManager scm;
+  private NodeManager nodeManager;
+  private ScmBlockLocationProtocolServerSideTranslatorPB service;
+  private final int nodeCount = 10;
+
+  @Before
+  public void setUp() throws Exception {
+    config = new OzoneConfiguration();
+    File dir = GenericTestUtils.getRandomizedTestDir();
+    config.set(HddsConfigKeys.OZONE_METADATA_DIRS, dir.toString());
+    SCMConfigurator configurator = new SCMConfigurator();
+    scm = TestUtils.getScm(config, configurator);
+    scm.start();
+    scm.exitSafeMode();
+    // add nodes to scm node manager
+    nodeManager = scm.getScmNodeManager();
+    for (int i = 0; i < nodeCount; i++) {
+      nodeManager.register(TestUtils.randomDatanodeDetails(), null, null);
+
+    }
+    server = scm.getBlockProtocolServer();
+    service = new ScmBlockLocationProtocolServerSideTranslatorPB(server);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (scm != null) {
+      scm.stop();
+      scm.join();
+    }
+  }
+
+  @Test
+  public void testSortDatanodes() throws Exception {
+    List<String> nodes = new ArrayList();
+    nodeManager.getAllNodes().stream().forEach(
+        node -> nodes.add(node.getNetworkName()));
+
+    // sort normal datanodes
+    String client;
+    client = nodes.get(0);
+    List<DatanodeDetails> datanodeDetails =
+        server.sortDatanodes(nodes, client);
+    System.out.println("client = " + client);
+    datanodeDetails.stream().forEach(
+        node -> System.out.println(node.toString()));
+    Assert.assertTrue(datanodeDetails.size() == nodeCount);
+
+    // illegal client 1
+    client += "X";
+    datanodeDetails = server.sortDatanodes(nodes, client);
+    System.out.println("client = " + client);
+    datanodeDetails.stream().forEach(
+        node -> System.out.println(node.toString()));
+    Assert.assertTrue(datanodeDetails.size() == nodeCount);
+    // illegal client 2
+    client = "/default-rack";
+    datanodeDetails = server.sortDatanodes(nodes, client);
+    System.out.println("client = " + client);
+    datanodeDetails.stream().forEach(
+        node -> System.out.println(node.toString()));
+    Assert.assertTrue(datanodeDetails.size() == nodeCount);
+
+    // illegal nodes to sort 1
+    nodes.add("/default-rack");
+    ScmBlockLocationProtocolProtos.SortDatanodesRequestProto request =
+        ScmBlockLocationProtocolProtos.SortDatanodesRequestProto
+            .newBuilder()
+            .addAllNodeNetworkName(nodes)
+            .setClient(client)
+            .build();
+    ScmBlockLocationProtocolProtos.SortDatanodesResponseProto resp =
+        service.sortDatanodes(request);
+    Assert.assertTrue(resp.getNodeList().size() == nodeCount);
+    System.out.println("client = " + client);
+    resp.getNodeList().stream().forEach(
+        node -> System.out.println(node.getNetworkName()));
+
+    // illegal nodes to sort 2
+    nodes.remove("/default-rack");
+    nodes.add(nodes.get(0) + "X");
+    request = ScmBlockLocationProtocolProtos.SortDatanodesRequestProto
+            .newBuilder()
+            .addAllNodeNetworkName(nodes)
+            .setClient(client)
+            .build();
+    resp = service.sortDatanodes(request);
+    Assert.assertTrue(resp.getNodeList().size() == nodeCount);
+    System.out.println("client = " + client);
+    resp.getNodeList().stream().forEach(
+        node -> System.out.println(node.getNetworkName()));
+
+    // all illegal nodes
+    nodes.clear();
+    nodes.add("/default-rack");
+    nodes.add("/default-rack-1");
+    nodes.add("/default-rack-2");
+    request = ScmBlockLocationProtocolProtos.SortDatanodesRequestProto
+        .newBuilder()
+        .addAllNodeNetworkName(nodes)
+        .setClient(client)
+        .build();
+    resp = service.sortDatanodes(request);
+    System.out.println("client = " + client);
+    Assert.assertTrue(resp.getNodeList().size() == 0);
+    resp.getNodeList().stream().forEach(
+        node -> System.out.println(node.getNetworkName()));
+  }
+}