Browse Source

HDFS-10928. Ozone:SCM: Support MXBean for SCM and NodeManager. Contributed by Weiwei Yang.

Anu Engineer 8 năm trước cách đây
mục cha
commit
5e95bdeef2

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/SCMMXBean.java

@@ -0,0 +1,52 @@
+/**
+ * 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.ozone.scm;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.scm.node.SCMNodeManager;
+
+import java.util.Map;
+
+/**
+ *
+ * This is the JMX management interface for scm information.
+ */
+@InterfaceAudience.Private
+public interface SCMMXBean {
+
+  /**
+   * Get the number of data nodes that in all states,
+   * valid states are defined by {@link SCMNodeManager.NODESTATE}.
+   *
+   * @return A state to number of nodes that in this state mapping
+   */
+  public Map<String, Integer> getNodeCount();
+
+  /**
+   * Get the SCM RPC server port that used to listen to datanode requests.
+   * @return SCM datanode RPC server port
+   */
+  public String getDatanodeRpcPort();
+
+  /**
+   * Get the SCM RPC server port that used to listen to client requests.
+   * @return SCM client RPC server port
+   */
+  public String getClientRpcPort();
+}

+ 44 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.scm.protocol.LocatedContainer;
@@ -73,12 +74,15 @@ import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.management.ObjectName;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 import java.util.UUID;
+import java.util.Map;
+import java.util.HashMap;
 
 import static org.apache.hadoop.ozone.OzoneConfigKeys
     .OZONE_SCM_CLIENT_ADDRESS_KEY;
@@ -107,7 +111,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
 @InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"})
 public class StorageContainerManager
     implements StorageContainerDatanodeProtocol,
-    StorageContainerLocationProtocol {
+    StorageContainerLocationProtocol, SCMMXBean {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(StorageContainerManager.class);
@@ -126,6 +130,9 @@ public class StorageContainerManager
   private final RPC.Server clientRpcServer;
   private final InetSocketAddress clientRpcAddress;
 
+  /** SCM mxbean*/
+  private ObjectName scmInfoBeanName;
+
   /**
    * Creates a new StorageContainerManager.  Configuration will be updated with
    * information on the actual listening addresses used for RPC servers.
@@ -161,7 +168,6 @@ public class StorageContainerManager
     datanodeRpcAddress = updateListenAddress(conf,
         OZONE_SCM_DATANODE_ADDRESS_KEY, datanodeRpcAddr, datanodeRpcServer);
 
-
     BlockingService storageProtoPbService =
         StorageContainerLocationProtocolProtos
             .StorageContainerLocationProtocolService
@@ -176,6 +182,8 @@ public class StorageContainerManager
         handlerCount);
     clientRpcAddress = updateListenAddress(conf,
         OZONE_SCM_CLIENT_ADDRESS_KEY, scmAddress, clientRpcServer);
+
+    registerMXBean();
   }
 
   /**
@@ -222,6 +230,18 @@ public class StorageContainerManager
     return rpcServer;
   }
 
+  private void registerMXBean() {
+    this.scmInfoBeanName = MBeans.register("StorageContainerManager",
+        "StorageContainerManagerInfo", this);
+  }
+
+  private void unregisterMXBean() {
+    if(this.scmInfoBeanName != null) {
+      MBeans.unregister(this.scmInfoBeanName);
+      this.scmInfoBeanName = null;
+    }
+  }
+
   /**
    * After starting an RPC server, updates configuration with the actual
    * listening address of that server. The listening address may be different
@@ -334,6 +354,12 @@ public class StorageContainerManager
     return clientRpcAddress;
   }
 
+  @Override
+  public String getClientRpcPort() {
+    InetSocketAddress addr = getClientRpcAddress();
+    return addr == null ? "0" : Integer.toString(addr.getPort());
+  }
+
   /**
    * Returns listening address of StorageDatanode Protocol RPC server.
    *
@@ -343,6 +369,12 @@ public class StorageContainerManager
     return datanodeRpcAddress;
   }
 
+  @Override
+  public String getDatanodeRpcPort() {
+    InetSocketAddress addr = getDatanodeRpcAddress();
+    return addr == null ? "0" : Integer.toString(addr.getPort());
+  }
+
   /**
    * Start service.
    */
@@ -363,6 +395,7 @@ public class StorageContainerManager
     clientRpcServer.stop();
     LOG.info("Stopping the RPC server for DataNodes");
     datanodeRpcServer.stop();
+    unregisterMXBean();
   }
 
   /**
@@ -437,6 +470,15 @@ public class StorageContainerManager
     return scmNodeManager.getNodeCount(nodestate);
   }
 
+  @Override
+  public Map<String, Integer> getNodeCount() {
+    Map<String, Integer> countMap = new HashMap<String, Integer>();
+    for (SCMNodeManager.NODESTATE state : SCMNodeManager.NODESTATE.values()) {
+      countMap.put(state.toString(), scmNodeManager.getNodeCount(state));
+    }
+    return countMap;
+  }
+
   /**
    * Returns node manager.
    * @return - Node Manager

+ 2 - 29
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/NodeManager.java

@@ -46,8 +46,8 @@ import java.util.List;
  * DECOMMISSIONED - Someone told us to remove this node from the tracking
  * list, by calling removeNode. We will throw away this nodes info soon.
  */
-public interface NodeManager extends StorageContainerNodeProtocol, Closeable,
-    Runnable {
+public interface NodeManager extends StorageContainerNodeProtocol,
+    NodeManagerMXBean, Closeable, Runnable {
   /**
    * Removes a data node from the management of this Node Manager.
    *
@@ -78,20 +78,6 @@ public interface NodeManager extends StorageContainerNodeProtocol, Closeable,
    */
   List<DatanodeID> getAllNodes();
 
-  /**
-   * Get the minimum number of nodes to get out of chill mode.
-   *
-   * @return int
-   */
-  int getMinimumChillModeNodes();
-
-  /**
-   * Reports if we have exited out of chill mode by discovering enough nodes.
-   *
-   * @return True if we are out of Node layer chill mode, false otherwise.
-   */
-  boolean isOutOfNodeChillMode();
-
   /**
    * Chill mode is the period when node manager waits for a minimum
    * configured number of datanodes to report in. This is called chill mode
@@ -112,19 +98,6 @@ public interface NodeManager extends StorageContainerNodeProtocol, Closeable,
    */
   void clearChillModeFlag();
 
-  /**
-   * Returns a chill mode status string.
-   * @return String
-   */
-  String getChillModeStatus();
-
-  /**
-   * Returns the status of manual chill mode flag.
-   * @return true if forceEnterChillMode has been called,
-   * false if forceExitChillMode or status is not set. eg. clearChillModeFlag.
-   */
-  boolean isInManualChillMode();
-
   /**
    * Enum that represents the Node State. This is used in calls to getNodeList
    * and getNodeCount. TODO: Add decommission when we support it.

+ 66 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/NodeManagerMXBean.java

@@ -0,0 +1,66 @@
+/**
+ * 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.ozone.scm.node;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.util.Map;
+
+/**
+ *
+ * This is the JMX management interface for node manager information.
+ */
+@InterfaceAudience.Private
+public interface NodeManagerMXBean {
+  /**
+   * Get the minimum number of nodes to get out of chill mode.
+   *
+   * @return int
+   */
+  public int getMinimumChillModeNodes();
+
+  /**
+   * Reports if we have exited out of chill mode by discovering enough nodes.
+   *
+   * @return True if we are out of Node layer chill mode, false otherwise.
+   */
+  public boolean isOutOfNodeChillMode();
+
+  /**
+   * Returns a chill mode status string.
+   * @return String
+   */
+  public String getChillModeStatus();
+
+
+  /**
+   * Returns the status of manual chill mode flag.
+   * @return true if forceEnterChillMode has been called,
+   * false if forceExitChillMode or status is not set. eg. clearChillModeFlag.
+   */
+  public boolean isInManualChillMode();
+
+  /**
+   * Get the number of data nodes that in all states,
+   * valid states are defined by {@link SCMNodeManager.NODESTATE}.
+   *
+   * @return A state to number of nodes that in this state mapping
+   */
+  public Map<String, Integer> getNodeCount();
+}

+ 28 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/SCMNodeManager.java

@@ -25,6 +25,7 @@ import org.apache.commons.collections.map.HashedMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.OzoneClientUtils;
 import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
@@ -46,6 +47,7 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.management.ObjectName;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
@@ -123,7 +125,8 @@ public class SCMNodeManager
   private final VersionInfo version;
   private Optional<Boolean> inManualChillMode;
   private final CommandQueue commandQueue;
-
+  // Node manager MXBean
+  private ObjectName nmInfoBean;
   /**
    * Constructs SCM machine Manager.
    */
@@ -162,6 +165,20 @@ public class SCMNodeManager
     Preconditions.checkState(heartbeatCheckerIntervalMs > 0);
     executorService.schedule(this, heartbeatCheckerIntervalMs,
         TimeUnit.MILLISECONDS);
+
+    registerMXBean();
+  }
+
+  private void registerMXBean() {
+    this.nmInfoBean = MBeans.register("SCMNodeManager",
+        "SCMNodeManagerInfo", this);
+  }
+
+  private void unregisterMXBean() {
+    if(this.nmInfoBean != null) {
+      MBeans.unregister(this.nmInfoBean);
+      this.nmInfoBean = null;
+    }
   }
 
   /**
@@ -595,6 +612,7 @@ public class SCMNodeManager
   @Override
   public void close() throws IOException {
     executorService.shutdown();
+    unregisterMXBean();
     try {
       if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
         executorService.shutdownNow();
@@ -729,4 +747,13 @@ public class SCMNodeManager
     return nodeStats.entrySet().stream().map(
         entry -> nodeStats.get(entry.getKey())).collect(Collectors.toList());
   }
+
+  @Override
+  public Map<String, Integer> getNodeCount() {
+    Map<String, Integer> nodeCountMap = new HashMap<String, Integer>();
+    for(NodeManager.NODESTATE state : NodeManager.NODESTATE.values()) {
+      nodeCountMap.put(state.toString(), getNodeCount(state));
+    }
+    return nodeCountMap;
+  }
 }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java

@@ -106,6 +106,10 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
     scm.join();
   }
 
+  public StorageContainerManager getStorageContainerManager() {
+    return this.scm;
+  }
+
   /**
    * Creates an {@link OzoneClient} connected to this cluster's REST service.
    * Callers take ownership of the client and must close it when done.

+ 139 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java

@@ -0,0 +1,139 @@
+/*
+ * 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.ozone.scm;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.scm.node.NodeManager;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.Map;
+import java.util.Iterator;
+
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.TabularData;
+
+public class TestSCMMXBean {
+
+  public static final Log LOG = LogFactory.getLog(TestSCMMXBean.class);
+  private static int numOfDatanodes = 1;
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static StorageContainerManager scm;
+  private static MBeanServer mbs;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    conf = new OzoneConfiguration();
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .numDataNodes(numOfDatanodes)
+        .setHandlerType("distributed")
+        .build();
+    scm = cluster.getStorageContainerManager();
+    mbs = ManagementFactory.getPlatformMBeanServer();
+  }
+
+  @AfterClass
+  public static void shutdown() {
+    IOUtils.cleanup(null, cluster);
+  }
+
+  @Test
+  public void testSCMMXBean() throws Exception {
+    ObjectName bean = new ObjectName(
+        "Hadoop:service=StorageContainerManager,"
+            + "name=StorageContainerManagerInfo");
+
+    String dnRpcPort = (String)mbs.getAttribute(bean,
+        "DatanodeRpcPort");
+    assertEquals(scm.getDatanodeRpcPort(), dnRpcPort);
+
+    String clientRpcPort = (String)mbs.getAttribute(bean,
+        "ClientRpcPort");
+    assertEquals(scm.getClientRpcPort(), clientRpcPort);
+
+    TabularData nodeCountObj = (TabularData)mbs.getAttribute(bean,
+        "NodeCount");
+    verifyEquals(nodeCountObj, scm.getNodeCount());
+  }
+
+  @Test
+  public void testSCMNodeManagerMXBean() throws Exception {
+    final NodeManager scmNm = scm.getScmNodeManager();
+    ObjectName bean = new ObjectName(
+        "Hadoop:service=SCMNodeManager,name=SCMNodeManagerInfo");
+
+    Integer minChillNodes = (Integer)mbs.getAttribute(bean,
+        "MinimumChillModeNodes");
+    assertEquals(scmNm.getMinimumChillModeNodes(),
+        minChillNodes.intValue());
+
+    boolean isOutOfChillMode = (boolean)mbs.getAttribute(bean,
+        "OutOfNodeChillMode");
+    assertEquals(scmNm.isOutOfNodeChillMode(), isOutOfChillMode);
+
+    String chillStatus = (String)mbs.getAttribute(bean,
+        "ChillModeStatus");
+    assertEquals(scmNm.getChillModeStatus(), chillStatus);
+
+    boolean inManualChillMode = (boolean)mbs.getAttribute(bean,
+        "InManualChillMode");
+    assertEquals(scmNm.isInManualChillMode(), inManualChillMode);
+
+    TabularData nodeCountObj = (TabularData)mbs.getAttribute(bean,
+        "NodeCount");
+    verifyEquals(nodeCountObj, scm.getScmNodeManager().getNodeCount());
+  }
+
+  /**
+   * An internal function used to compare a TabularData returned
+   * by JMX with the expected data in a Map.
+   */
+  private void verifyEquals(TabularData data1,
+      Map<String, Integer> data2) {
+    if (data1 == null || data2 == null) {
+      fail("Data should not be null.");
+    }
+    for (Object obj : data1.values()) {
+      // Each TabularData is a set of CompositeData
+      assertTrue(obj instanceof CompositeData);
+      CompositeData cds = (CompositeData) obj;
+      assertEquals(2, cds.values().size());
+      Iterator<?> it = cds.values().iterator();
+      String key = it.next().toString();
+      String value = it.next().toString();
+      int num = Integer.parseInt(value);
+      assertTrue(data2.containsKey(key));
+      assertEquals(data2.get(key).intValue(), num);
+    }
+  }
+}

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/MockNodeManager.java

@@ -30,8 +30,10 @@ import org.apache.hadoop.ozone.scm.node.NodeManager;
 import org.apache.hadoop.ozone.scm.node.SCMNodeStat;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Test Helper for testing container Mapping.
@@ -263,4 +265,13 @@ public class MockNodeManager implements NodeManager {
       SCMNodeReport nodeReport) {
     return null;
   }
+
+  @Override
+  public Map<String, Integer> getNodeCount() {
+    Map<String, Integer> nodeCountMap = new HashMap<String, Integer>();
+    for(NodeManager.NODESTATE state : NodeManager.NODESTATE.values()) {
+      nodeCountMap.put(state.toString(), getNodeCount(state));
+    }
+    return nodeCountMap;
+  }
 }