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 fe...@apache.org on 2022/04/24 09:18:13 UTC

[hadoop] 01/01: Revert "HDFS-16488. [SPS]: Expose metrics to JMX for external SPS (#4035)"

This is an automated email from the ASF dual-hosted git repository.

ferhui pushed a commit to branch revert-4035-HDFS-16488
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit cd13853001f6b5b2121ade16b3e6aab6933864d8
Author: Hui Fei <fe...@apache.org>
AuthorDate: Sun Apr 24 17:18:03 2022 +0800

    Revert "HDFS-16488. [SPS]: Expose metrics to JMX for external SPS (#4035)"
    
    This reverts commit acc0e0a2101cf9e75bae0710b75f4e88d01f2e14.
---
 .../sps/BlockStorageMovementAttemptedItems.java    |  10 ---
 .../namenode/sps/StoragePolicySatisfier.java       |   4 +-
 .../hadoop/hdfs/server/sps/ExternalSPSContext.java |  17 ----
 .../server/sps/ExternalStoragePolicySatisfier.java |  12 +--
 .../server/sps/metrics/ExternalSPSBeanMetrics.java | 100 ---------------------
 .../hdfs/server/sps/metrics/ExternalSPSMXBean.java |  52 -----------
 .../hdfs/server/sps/metrics/package-info.java      |  22 -----
 .../sps/TestExternalStoragePolicySatisfier.java    |  34 -------
 8 files changed, 5 insertions(+), 246 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
index 174f2be7a71..1a136e571d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementAttemptedItems.java
@@ -301,16 +301,6 @@ public class BlockStorageMovementAttemptedItems {
     }
   }
 
-  @VisibleForTesting
-  public List<AttemptedItemInfo> getStorageMovementAttemptedItems() {
-    return storageMovementAttemptedItems;
-  }
-
-  @VisibleForTesting
-  public BlockingQueue<Block> getMovementFinishedBlocks() {
-    return movementFinishedBlocks;
-  }
-
   public void clearQueues() {
     movementFinishedBlocks.clear();
     synchronized (storageMovementAttemptedItems) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
index 3efe6b1cd6d..47596019af4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfier.java
@@ -1077,7 +1077,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
    * attempted or reported time stamp. This is used by
    * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
    */
-  public final static class AttemptedItemInfo extends ItemInfo {
+  final static class AttemptedItemInfo extends ItemInfo {
     private long lastAttemptedOrReportedTime;
     private final Set<Block> blocks;
 
@@ -1095,7 +1095,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
      * @param retryCount
      *          file retry count
      */
-    public AttemptedItemInfo(long rootId, long trackId,
+    AttemptedItemInfo(long rootId, long trackId,
         long lastAttemptedOrReportedTime,
         Set<Block> blocks, int retryCount) {
       super(rootId, trackId, retryCount);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
index 78f1dc6bf3e..8427e93a709 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSContext.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -40,12 +39,10 @@ import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.Context;
 import org.apache.hadoop.hdfs.server.namenode.sps.FileCollector;
 import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeMap;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier.DatanodeWithStorage;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.hdfs.server.sps.metrics.ExternalSPSBeanMetrics;
 import org.apache.hadoop.net.NetworkTopology;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -65,7 +62,6 @@ public class ExternalSPSContext implements Context {
   private final FileCollector fileCollector;
   private final BlockMoveTaskHandler externalHandler;
   private final BlockMovementListener blkMovementListener;
-  private ExternalSPSBeanMetrics spsBeanMetrics;
 
   public ExternalSPSContext(SPSService service, NameNodeConnector nnc) {
     this.service = service;
@@ -212,17 +208,4 @@ public class ExternalSPSContext implements Context {
       LOG.info("Movement attempted blocks", actualBlockMovements);
     }
   }
-
-  public void initMetrics(StoragePolicySatisfier sps) {
-    spsBeanMetrics = new ExternalSPSBeanMetrics(sps);
-  }
-
-  public void closeMetrics() {
-    spsBeanMetrics.close();
-  }
-
-  @VisibleForTesting
-  public ExternalSPSBeanMetrics getSpsBeanMetrics() {
-    return spsBeanMetrics;
-  }
 }
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
index 40d01bda4a3..efbee7c030f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java
@@ -48,7 +48,8 @@ import org.slf4j.LoggerFactory;
  */
 @InterfaceAudience.Private
 public final class ExternalStoragePolicySatisfier {
-  public static final Logger LOG = LoggerFactory.getLogger(ExternalStoragePolicySatisfier.class);
+  public static final Logger LOG = LoggerFactory
+      .getLogger(ExternalStoragePolicySatisfier.class);
 
   private ExternalStoragePolicySatisfier() {
     // This is just a class to start and run external sps.
@@ -59,7 +60,6 @@ public final class ExternalStoragePolicySatisfier {
    */
   public static void main(String[] args) throws Exception {
     NameNodeConnector nnc = null;
-    ExternalSPSContext context = null;
     try {
       StringUtils.startupShutdownMessage(StoragePolicySatisfier.class, args,
           LOG);
@@ -69,10 +69,9 @@ public final class ExternalStoragePolicySatisfier {
       StoragePolicySatisfier sps = new StoragePolicySatisfier(spsConf);
       nnc = getNameNodeConnector(spsConf);
 
-      context = new ExternalSPSContext(sps, nnc);
+      ExternalSPSContext context = new ExternalSPSContext(sps, nnc);
       sps.init(context);
       sps.start(StoragePolicySatisfierMode.EXTERNAL);
-      context.initMetrics(sps);
       if (sps != null) {
         sps.join();
       }
@@ -83,11 +82,6 @@ public final class ExternalStoragePolicySatisfier {
       if (nnc != null) {
         nnc.close();
       }
-      if (context!= null) {
-        if (context.getSpsBeanMetrics() != null) {
-          context.closeMetrics();
-        }
-      }
     }
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSBeanMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSBeanMetrics.java
deleted file mode 100644
index 75546386f9d..00000000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSBeanMetrics.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * 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.hdfs.server.sps.metrics;
-
-import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.sps.ItemInfo;
-import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
-import org.apache.hadoop.metrics2.util.MBeans;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.management.NotCompliantMBeanException;
-import javax.management.ObjectName;
-import javax.management.StandardMBean;
-import java.util.HashSet;
-
-/**
- * Expose the ExternalSPS metrics.
- */
-public class ExternalSPSBeanMetrics implements ExternalSPSMXBean {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ExternalSPSBeanMetrics.class);
-
-  /**
-   * ExternalSPS bean.
-   */
-  private ObjectName externalSPSBeanName;
-  private StoragePolicySatisfier storagePolicySatisfier;
-
-  public ExternalSPSBeanMetrics(StoragePolicySatisfier sps) {
-    try {
-      this.storagePolicySatisfier = sps;
-      StandardMBean bean = new StandardMBean(this, ExternalSPSMXBean.class);
-      this.externalSPSBeanName = MBeans.register("ExternalSPS", "ExternalSPS", bean);
-      LOG.info("Registered ExternalSPS MBean: {}", this.externalSPSBeanName);
-    } catch (NotCompliantMBeanException e) {
-      throw new RuntimeException("Bad externalSPS MBean setup", e);
-    }
-  }
-
-  /**
-   * Unregister the JMX interfaces.
-   */
-  public void close() {
-    if (externalSPSBeanName != null) {
-      MBeans.unregister(externalSPSBeanName);
-      externalSPSBeanName = null;
-    }
-  }
-
-  @Override
-  public int getProcessingQueueSize() {
-    return storagePolicySatisfier.processingQueueSize();
-  }
-
-  @VisibleForTesting
-  public void updateProcessingQueueSize() {
-    storagePolicySatisfier.getStorageMovementQueue()
-        .add(new ItemInfo(0, 1, 1));
-  }
-
-  @Override
-  public int getMovementFinishedBlocksCount() {
-    return storagePolicySatisfier.getAttemptedItemsMonitor().getMovementFinishedBlocksCount();
-  }
-
-  @VisibleForTesting
-  public void updateMovementFinishedBlocksCount() {
-    storagePolicySatisfier.getAttemptedItemsMonitor().getMovementFinishedBlocks()
-        .add(new Block(1));
-  }
-
-  @Override
-  public int getAttemptedItemsCount() {
-    return storagePolicySatisfier.getAttemptedItemsMonitor().getAttemptedItemsCount();
-  }
-
-  @VisibleForTesting
-  public void updateAttemptedItemsCount() {
-    storagePolicySatisfier.getAttemptedItemsMonitor().getStorageMovementAttemptedItems()
-        .add(new StoragePolicySatisfier.AttemptedItemInfo(0, 1, 1, new HashSet<>(), 1));
-  }
-}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSMXBean.java
deleted file mode 100644
index 12492fbbf0d..00000000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSMXBean.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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.hdfs.server.sps.metrics;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This is the JMX management interface for ExternalSPS information.
- * End users shouldn't be implementing these interfaces, and instead
- * access this information through the JMX APIs.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public interface ExternalSPSMXBean {
-
-  /**
-   * Gets the queue size of StorageMovementNeeded.
-   *
-   * @return the queue size of StorageMovementNeeded.
-   */
-  int getProcessingQueueSize();
-
-  /**
-   * Gets the count of movement finished blocks.
-   *
-   * @return the count of movement finished blocks.
-   */
-  int getMovementFinishedBlocksCount();
-
-  /**
-   * Gets the count of attempted items.
-   *
-   * @return the count of attempted items.
-   */
-  int getAttemptedItemsCount();
-}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/package-info.java
deleted file mode 100644
index a6d26f69d37..00000000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * 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.
- */
-
-/**
- * This package provides the ability to expose external SPS metrics to JMX.
- */
-package org.apache.hadoop.hdfs.server.sps.metrics;
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
index be657d0352e..2559836a0a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java
@@ -44,7 +44,6 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.lang.management.ManagementFactory;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -85,7 +84,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockMovementListener;
 import org.apache.hadoop.hdfs.server.namenode.sps.BlockStorageMovementAttemptedItems;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
-import org.apache.hadoop.hdfs.server.sps.metrics.ExternalSPSBeanMetrics;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.SecurityUtil;
@@ -104,8 +102,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
 import java.util.function.Supplier;
 
 /**
@@ -1821,34 +1817,4 @@ public class TestExternalStoragePolicySatisfier {
       actualBlockMovements.clear();
     }
   }
-
-  @Test(timeout = 300000)
-  public void testExternalSPSMetrics() throws Exception {
-    try {
-      createCluster();
-      // Start JMX but stop SPS thread to prevent mock data from being consumed.
-      externalSps.stop(true);
-      externalCtxt.initMetrics(externalSps);
-
-      ExternalSPSBeanMetrics spsBeanMetrics = externalCtxt.getSpsBeanMetrics();
-      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-      ObjectName mxBeanName = new ObjectName("Hadoop:service=ExternalSPS,name=ExternalSPS");
-      // Assert metrics before update.
-      assertEquals(0, mbs.getAttribute(mxBeanName, "AttemptedItemsCount"));
-      assertEquals(0, mbs.getAttribute(mxBeanName, "ProcessingQueueSize"));
-      assertEquals(0, mbs.getAttribute(mxBeanName, "MovementFinishedBlocksCount"));
-
-      // Update metrics.
-      spsBeanMetrics.updateAttemptedItemsCount();
-      spsBeanMetrics.updateProcessingQueueSize();
-      spsBeanMetrics.updateMovementFinishedBlocksCount();
-
-      // Assert metrics after update.
-      assertEquals(1, mbs.getAttribute(mxBeanName, "AttemptedItemsCount"));
-      assertEquals(1, mbs.getAttribute(mxBeanName, "ProcessingQueueSize"));
-      assertEquals(1, mbs.getAttribute(mxBeanName, "MovementFinishedBlocksCount"));
-    } finally {
-      shutdownCluster();
-    }
-  }
 }


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