浏览代码

HDFS-16477. [SPS]: Add metric PendingSPSPaths for getting the number of paths to be processed by SPS (#4009). Contributed by tomscut.

Signed-off-by: Ayush Saxena <ayushsaxena@apache.org>
litao 3 年之前
父节点
当前提交
34b3275bf4
共有 17 个文件被更改,包括 137 次插入25 次删除
  1. 1 0
      hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
  2. 7 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
  3. 10 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
  4. 6 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java
  5. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
  6. 15 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
  7. 2 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
  8. 4 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java
  9. 10 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
  10. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto
  11. 5 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestRBFMetrics.java
  12. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
  13. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  14. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  15. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
  16. 19 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
  17. 34 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java

+ 1 - 0
hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md

@@ -299,6 +299,7 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `FSN(Read/Write)Lock`*OperationName*`NanosAvgTime` | Average time of holding the lock by operations in nanoseconds |
 | `FSN(Read/Write)Lock`*OperationName*`NanosAvgTime` | Average time of holding the lock by operations in nanoseconds |
 | `FSN(Read/Write)LockOverallNanosNumOps`  | Total number of acquiring lock by all operations |
 | `FSN(Read/Write)LockOverallNanosNumOps`  | Total number of acquiring lock by all operations |
 | `FSN(Read/Write)LockOverallNanosAvgTime` | Average time of holding the lock by all operations in nanoseconds |
 | `FSN(Read/Write)LockOverallNanosAvgTime` | Average time of holding the lock by all operations in nanoseconds |
+| `PendingSPSPaths` | The number of paths to be processed by storage policy satisfier |
 
 
 JournalNode
 JournalNode
 -----------
 -----------

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java

@@ -343,4 +343,11 @@ public interface FederationMBean {
    * with the highest risk of loss.
    * with the highest risk of loss.
    */
    */
   long getHighestPriorityLowRedundancyECBlocks();
   long getHighestPriorityLowRedundancyECBlocks();
+
+  /**
+   * Returns the number of paths to be processed by storage policy satisfier.
+   *
+   * @return The number of paths to be processed by sps.
+   */
+  int getPendingSPSPaths();
 }
 }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java

@@ -874,6 +874,16 @@ public class NamenodeBeanMetrics
     return 0;
     return 0;
   }
   }
 
 
+  @Override
+  public int getPendingSPSPaths() {
+    try {
+      return getRBFMetrics().getPendingSPSPaths();
+    } catch (IOException e) {
+      LOG.debug("Failed to get number of paths to be processed by sps", e);
+    }
+    return 0;
+  }
+
   private Router getRouter() throws IOException {
   private Router getRouter() throws IOException {
     if (this.router == null) {
     if (this.router == null) {
       throw new IOException("Router is not initialized");
       throw new IOException("Router is not initialized");

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java

@@ -746,6 +746,12 @@ public class RBFMetrics implements RouterMBean, FederationMBean {
         MembershipStats::getHighestPriorityLowRedundancyECBlocks);
         MembershipStats::getHighestPriorityLowRedundancyECBlocks);
   }
   }
 
 
+  @Override
+  public int getPendingSPSPaths() {
+    return getNameserviceAggregatedInt(
+        MembershipStats::getPendingSPSPaths);
+  }
+
   @Override
   @Override
   @Metric({"RouterFederationRenameCount", "Number of federation rename"})
   @Metric({"RouterFederationRenameCount", "Number of federation rename"})
   public int getRouterFederationRenameCount() {
   public int getRouterFederationRenameCount() {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java

@@ -306,6 +306,7 @@ public class MembershipNamenodeResolver
           report.getHighestPriorityLowRedundancyReplicatedBlocks());
           report.getHighestPriorityLowRedundancyReplicatedBlocks());
       stats.setHighestPriorityLowRedundancyECBlocks(
       stats.setHighestPriorityLowRedundancyECBlocks(
           report.getHighestPriorityLowRedundancyECBlocks());
           report.getHighestPriorityLowRedundancyECBlocks());
+      stats.setPendingSPSPaths(report.getPendingSPSPaths());
       record.setStats(stats);
       record.setStats(stats);
     }
     }
 
 

+ 15 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java

@@ -75,6 +75,7 @@ public class NamenodeStatusReport {
   private long numberOfMissingBlocksWithReplicationFactorOne = -1;
   private long numberOfMissingBlocksWithReplicationFactorOne = -1;
   private long highestPriorityLowRedundancyReplicatedBlocks = -1;
   private long highestPriorityLowRedundancyReplicatedBlocks = -1;
   private long highestPriorityLowRedundancyECBlocks = -1;
   private long highestPriorityLowRedundancyECBlocks = -1;
+  private int pendingSPSPaths = -1;
 
 
   /** If the fields are valid. */
   /** If the fields are valid. */
   private boolean registrationValid = false;
   private boolean registrationValid = false;
@@ -367,12 +368,13 @@ public class NamenodeStatusReport {
    * @param numBlocksPendingReplication Number of blocks pending replication.
    * @param numBlocksPendingReplication Number of blocks pending replication.
    * @param numBlocksUnderReplicated Number of blocks under replication.
    * @param numBlocksUnderReplicated Number of blocks under replication.
    * @param numBlocksPendingDeletion Number of blocks pending deletion.
    * @param numBlocksPendingDeletion Number of blocks pending deletion.
-   * @param providedSpace Space in provided storage.
+   * @param providedStorageSpace Space in provided storage.
+   * @param numPendingSPSPaths The number of paths to be processed by storage policy satisfier.
    */
    */
   public void setNamesystemInfo(long available, long total,
   public void setNamesystemInfo(long available, long total,
       long numFiles, long numBlocks, long numBlocksMissing,
       long numFiles, long numBlocks, long numBlocksMissing,
       long numBlocksPendingReplication, long numBlocksUnderReplicated,
       long numBlocksPendingReplication, long numBlocksUnderReplicated,
-      long numBlocksPendingDeletion, long providedSpace) {
+      long numBlocksPendingDeletion, long providedStorageSpace, int numPendingSPSPaths) {
     this.totalSpace = total;
     this.totalSpace = total;
     this.availableSpace = available;
     this.availableSpace = available;
     this.numOfBlocks = numBlocks;
     this.numOfBlocks = numBlocks;
@@ -382,7 +384,8 @@ public class NamenodeStatusReport {
     this.numOfBlocksPendingDeletion = numBlocksPendingDeletion;
     this.numOfBlocksPendingDeletion = numBlocksPendingDeletion;
     this.numOfFiles = numFiles;
     this.numOfFiles = numFiles;
     this.statsValid = true;
     this.statsValid = true;
-    this.providedSpace = providedSpace;
+    this.providedSpace = providedStorageSpace;
+    this.pendingSPSPaths = numPendingSPSPaths;
   }
   }
 
 
   /**
   /**
@@ -460,6 +463,15 @@ public class NamenodeStatusReport {
     return this.highestPriorityLowRedundancyECBlocks;
     return this.highestPriorityLowRedundancyECBlocks;
   }
   }
 
 
+  /**
+   * Returns the number of paths to be processed by storage policy satisfier.
+   *
+   * @return The number of paths to be processed by sps.
+   */
+  public int getPendingSPSPaths() {
+    return this.pendingSPSPaths;
+  }
+
   /**
   /**
    * Get the number of blocks.
    * Get the number of blocks.
    *
    *

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java

@@ -478,7 +478,8 @@ public class NamenodeHeartbeatService extends PeriodicService {
               jsonObject.getLong("PendingReplicationBlocks"),
               jsonObject.getLong("PendingReplicationBlocks"),
               jsonObject.getLong("UnderReplicatedBlocks"),
               jsonObject.getLong("UnderReplicatedBlocks"),
               jsonObject.getLong("PendingDeletionBlocks"),
               jsonObject.getLong("PendingDeletionBlocks"),
-              jsonObject.optLong("ProvidedCapacityTotal"));
+              jsonObject.optLong("ProvidedCapacityTotal"),
+              jsonObject.getInt("PendingSPSPaths"));
         }
         }
       }
       }
     }
     }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java

@@ -133,6 +133,10 @@ public abstract class MembershipStats extends BaseRecord {
 
 
   public abstract long getHighestPriorityLowRedundancyECBlocks();
   public abstract long getHighestPriorityLowRedundancyECBlocks();
 
 
+  public abstract void setPendingSPSPaths(int pendingSPSPaths);
+
+  public abstract int getPendingSPSPaths();
+
   @Override
   @Override
   public SortedMap<String, String> getPrimaryKeys() {
   public SortedMap<String, String> getPrimaryKeys() {
     // This record is not stored directly, no key needed
     // This record is not stored directly, no key needed

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java

@@ -297,4 +297,14 @@ public class MembershipStatsPBImpl extends MembershipStats
     return this.translator.getProtoOrBuilder()
     return this.translator.getProtoOrBuilder()
         .getHighestPriorityLowRedundancyECBlocks();
         .getHighestPriorityLowRedundancyECBlocks();
   }
   }
+
+  @Override
+  public void setPendingSPSPaths(int pendingSPSPaths) {
+    this.translator.getBuilder().setPendingSPSPaths(pendingSPSPaths);
+  }
+
+  @Override
+  public int getPendingSPSPaths() {
+    return this.translator.getProtoOrBuilder().getPendingSPSPaths();
+  }
 }
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto

@@ -54,6 +54,7 @@ message NamenodeMembershipStatsRecordProto {
   optional uint64 numberOfMissingBlocksWithReplicationFactorOne = 31;
   optional uint64 numberOfMissingBlocksWithReplicationFactorOne = 31;
   optional uint64 highestPriorityLowRedundancyReplicatedBlocks = 32;
   optional uint64 highestPriorityLowRedundancyReplicatedBlocks = 32;
   optional uint64 HighestPriorityLowRedundancyECBlocks = 33;
   optional uint64 HighestPriorityLowRedundancyECBlocks = 33;
+  optional uint32 pendingSPSPaths = 34;
 }
 }
 
 
 message NamenodeMembershipRecordProto {
 message NamenodeMembershipRecordProto {

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestRBFMetrics.java

@@ -219,6 +219,8 @@ public class TestRBFMetrics extends TestMetricsBase {
           json.getLong("numOfEnteringMaintenanceDataNodes"));
           json.getLong("numOfEnteringMaintenanceDataNodes"));
       assertEquals(stats.getProvidedSpace(),
       assertEquals(stats.getProvidedSpace(),
           json.getLong("providedSpace"));
           json.getLong("providedSpace"));
+      assertEquals(stats.getPendingSPSPaths(),
+          json.getInt("pendingSPSPaths"));
       nameservicesFound++;
       nameservicesFound++;
     }
     }
     assertEquals(getNameservices().size(), nameservicesFound);
     assertEquals(getNameservices().size(), nameservicesFound);
@@ -296,6 +298,7 @@ public class TestRBFMetrics extends TestMetricsBase {
     long highestPriorityLowRedundancyReplicatedBlocks = 0;
     long highestPriorityLowRedundancyReplicatedBlocks = 0;
     long highestPriorityLowRedundancyECBlocks = 0;
     long highestPriorityLowRedundancyECBlocks = 0;
     long numFiles = 0;
     long numFiles = 0;
+    int pendingSPSPaths = 0;
     for (MembershipState mock : getActiveMemberships()) {
     for (MembershipState mock : getActiveMemberships()) {
       MembershipStats stats = mock.getStats();
       MembershipStats stats = mock.getStats();
       numBlocks += stats.getNumOfBlocks();
       numBlocks += stats.getNumOfBlocks();
@@ -316,6 +319,7 @@ public class TestRBFMetrics extends TestMetricsBase {
           stats.getHighestPriorityLowRedundancyReplicatedBlocks();
           stats.getHighestPriorityLowRedundancyReplicatedBlocks();
       highestPriorityLowRedundancyECBlocks +=
       highestPriorityLowRedundancyECBlocks +=
           stats.getHighestPriorityLowRedundancyECBlocks();
           stats.getHighestPriorityLowRedundancyECBlocks();
+      pendingSPSPaths += stats.getPendingSPSPaths();
     }
     }
 
 
     assertEquals(numBlocks, bean.getNumBlocks());
     assertEquals(numBlocks, bean.getNumBlocks());
@@ -342,6 +346,7 @@ public class TestRBFMetrics extends TestMetricsBase {
         bean.getHighestPriorityLowRedundancyReplicatedBlocks());
         bean.getHighestPriorityLowRedundancyReplicatedBlocks());
     assertEquals(highestPriorityLowRedundancyECBlocks,
     assertEquals(highestPriorityLowRedundancyECBlocks,
         bean.getHighestPriorityLowRedundancyECBlocks());
         bean.getHighestPriorityLowRedundancyECBlocks());
+    assertEquals(pendingSPSPaths, bean.getPendingSPSPaths());
   }
   }
 
 
   private void validateClusterStatsRouterBean(RouterMBean bean) {
   private void validateClusterStatsRouterBean(RouterMBean bean) {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java

@@ -269,6 +269,7 @@ public final class FederationStateStoreTestUtils {
     stats.setNumOfDecomActiveDatanodes(15);
     stats.setNumOfDecomActiveDatanodes(15);
     stats.setNumOfDecomDeadDatanodes(5);
     stats.setNumOfDecomDeadDatanodes(5);
     stats.setNumOfBlocks(10);
     stats.setNumOfBlocks(10);
+    stats.setPendingSPSPaths(10);
     entry.setStats(stats);
     entry.setStats(stats);
     return entry;
     return entry;
   }
   }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -298,6 +298,14 @@ public class BlockManager implements BlockStatsMXBean {
     return blocksMap.getECBlockGroups();
     return blocksMap.getECBlockGroups();
   }
   }
 
 
+  /** Used by metrics. */
+  public int getPendingSPSPaths() {
+    if (spsManager != null) {
+      return spsManager.getPendingSPSPaths();
+    }
+    return 0;
+  }
+
   /**
   /**
    * redundancyRecheckInterval is how often namenode checks for new
    * redundancyRecheckInterval is how often namenode checks for new
    * reconstruction work.
    * reconstruction work.

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4875,6 +4875,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         dtSecretManager.getCurrentTokensSize() : -1;
         dtSecretManager.getCurrentTokensSize() : -1;
   }
   }
 
 
+  @Override
+  @Metric({"PendingSPSPaths", "The number of paths to be processed by storage policy satisfier"})
+  public int getPendingSPSPaths() {
+    return blockManager.getPendingSPSPaths();
+  }
+
   /**
   /**
    * Returns the length of the wait Queue for the FSNameSystemLock.
    * Returns the length of the wait Queue for the FSNameSystemLock.
    *
    *

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java

@@ -254,4 +254,11 @@ public interface FSNamesystemMBean {
    * @return number of DTs
    * @return number of DTs
    */
    */
   long getCurrentTokensCount();
   long getCurrentTokensCount();
+
+  /**
+   * Returns the number of paths to be processed by storage policy satisfier.
+   *
+   * @return The number of paths to be processed by sps.
+   */
+  int getPendingSPSPaths();
 }
 }

+ 19 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java

@@ -60,7 +60,7 @@ public class StoragePolicySatisfyManager {
   private final StoragePolicySatisfier spsService;
   private final StoragePolicySatisfier spsService;
   private final boolean storagePolicyEnabled;
   private final boolean storagePolicyEnabled;
   private volatile StoragePolicySatisfierMode mode;
   private volatile StoragePolicySatisfierMode mode;
-  private final Queue<Long> pathsToBeTraveresed;
+  private final Queue<Long> pathsToBeTraversed;
   private final int outstandingPathsLimit;
   private final int outstandingPathsLimit;
   private final Namesystem namesystem;
   private final Namesystem namesystem;
 
 
@@ -77,7 +77,7 @@ public class StoragePolicySatisfyManager {
         DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY,
         DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY,
         DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT);
         DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT);
     mode = StoragePolicySatisfierMode.fromString(modeVal);
     mode = StoragePolicySatisfierMode.fromString(modeVal);
-    pathsToBeTraveresed = new LinkedList<Long>();
+    pathsToBeTraversed = new LinkedList<Long>();
     this.namesystem = namesystem;
     this.namesystem = namesystem;
     // instantiate SPS service by just keeps config reference and not starting
     // instantiate SPS service by just keeps config reference and not starting
     // any supporting threads.
     // any supporting threads.
@@ -218,8 +218,8 @@ public class StoragePolicySatisfyManager {
    *         storages.
    *         storages.
    */
    */
   public Long getNextPathId() {
   public Long getNextPathId() {
-    synchronized (pathsToBeTraveresed) {
-      return pathsToBeTraveresed.poll();
+    synchronized (pathsToBeTraversed) {
+      return pathsToBeTraversed.poll();
     }
     }
   }
   }
 
 
@@ -228,7 +228,7 @@ public class StoragePolicySatisfyManager {
    * @throws IOException
    * @throws IOException
    */
    */
   public void verifyOutstandingPathQLimit() throws IOException {
   public void verifyOutstandingPathQLimit() throws IOException {
-    long size = pathsToBeTraveresed.size();
+    long size = pathsToBeTraversed.size();
     // Checking that the SPS call Q exceeds the allowed limit.
     // Checking that the SPS call Q exceeds the allowed limit.
     if (outstandingPathsLimit - size <= 0) {
     if (outstandingPathsLimit - size <= 0) {
       LOG.debug("Satisifer Q - outstanding limit:{}, current size:{}",
       LOG.debug("Satisifer Q - outstanding limit:{}, current size:{}",
@@ -244,15 +244,15 @@ public class StoragePolicySatisfyManager {
    * @throws IOException
    * @throws IOException
    */
    */
   private void clearPathIds(){
   private void clearPathIds(){
-    synchronized (pathsToBeTraveresed) {
-      Iterator<Long> iterator = pathsToBeTraveresed.iterator();
+    synchronized (pathsToBeTraversed) {
+      Iterator<Long> iterator = pathsToBeTraversed.iterator();
       while (iterator.hasNext()) {
       while (iterator.hasNext()) {
         Long trackId = iterator.next();
         Long trackId = iterator.next();
         try {
         try {
           namesystem.removeXattr(trackId,
           namesystem.removeXattr(trackId,
               HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
               HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY);
         } catch (IOException e) {
         } catch (IOException e) {
-          LOG.debug("Failed to remove sps xatttr!", e);
+          LOG.debug("Failed to remove sps xattr!", e);
         }
         }
         iterator.remove();
         iterator.remove();
       }
       }
@@ -263,8 +263,8 @@ public class StoragePolicySatisfyManager {
    * Clean up all sps path ids.
    * Clean up all sps path ids.
    */
    */
   public void removeAllPathIds() {
   public void removeAllPathIds() {
-    synchronized (pathsToBeTraveresed) {
-      pathsToBeTraveresed.clear();
+    synchronized (pathsToBeTraversed) {
+      pathsToBeTraversed.clear();
     }
     }
   }
   }
 
 
@@ -273,8 +273,8 @@ public class StoragePolicySatisfyManager {
    * @param id
    * @param id
    */
    */
   public void addPathId(long id) {
   public void addPathId(long id) {
-    synchronized (pathsToBeTraveresed) {
-      pathsToBeTraveresed.add(id);
+    synchronized (pathsToBeTraversed) {
+      pathsToBeTraversed.add(id);
     }
     }
   }
   }
 
 
@@ -292,4 +292,11 @@ public class StoragePolicySatisfyManager {
   public StoragePolicySatisfierMode getMode() {
   public StoragePolicySatisfierMode getMode() {
     return mode;
     return mode;
   }
   }
+
+  /**
+   * @return the number of paths to be processed by storage policy satisfier.
+   */
+  public int getPendingSPSPaths() {
+    return pathsToBeTraversed.size();
+  }
 }
 }

+ 34 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java

@@ -36,6 +36,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
@@ -202,7 +203,15 @@ public class TestExternalStoragePolicySatisfier {
   private void createCluster(boolean createMoverPath) throws IOException {
   private void createCluster(boolean createMoverPath) throws IOException {
     getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES,
     setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES,
-        STORAGES_PER_DATANODE, CAPACITY, createMoverPath));
+        STORAGES_PER_DATANODE, CAPACITY, createMoverPath, true));
+    getFS();
+    writeContent(FILE);
+  }
+
+  private void createClusterDoNotStartSPS() throws IOException {
+    getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES,
+        STORAGES_PER_DATANODE, CAPACITY, true, false));
     getFS();
     getFS();
     writeContent(FILE);
     writeContent(FILE);
   }
   }
@@ -211,12 +220,12 @@ public class TestExternalStoragePolicySatisfier {
       StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
       StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
       long nodeCapacity) throws IOException {
       long nodeCapacity) throws IOException {
     return startCluster(conf, storageTypes, numberOfDatanodes, storagesPerDn,
     return startCluster(conf, storageTypes, numberOfDatanodes, storagesPerDn,
-        nodeCapacity, false);
+        nodeCapacity, false, true);
   }
   }
 
 
   private MiniDFSCluster startCluster(final Configuration conf,
   private MiniDFSCluster startCluster(final Configuration conf,
       StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
       StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
-      long nodeCapacity, boolean createMoverPath) throws IOException {
+      long nodeCapacity, boolean createMoverPath, boolean startSPS) throws IOException {
     long[][] capacities = new long[numberOfDatanodes][storagesPerDn];
     long[][] capacities = new long[numberOfDatanodes][storagesPerDn];
     for (int i = 0; i < numberOfDatanodes; i++) {
     for (int i = 0; i < numberOfDatanodes; i++) {
       for (int j = 0; j < storagesPerDn; j++) {
       for (int j = 0; j < storagesPerDn; j++) {
@@ -228,14 +237,16 @@ public class TestExternalStoragePolicySatisfier {
         .storageTypes(storageTypes).storageCapacities(capacities).build();
         .storageTypes(storageTypes).storageCapacities(capacities).build();
     cluster.waitActive();
     cluster.waitActive();
 
 
-    nnc = DFSTestUtil.getNameNodeConnector(getConf(),
-        HdfsServerConstants.MOVER_ID_PATH, 1, createMoverPath);
+    if (startSPS) {
+      nnc = DFSTestUtil.getNameNodeConnector(getConf(),
+          HdfsServerConstants.MOVER_ID_PATH, 1, createMoverPath);
 
 
-    externalSps = new StoragePolicySatisfier(getConf());
-    externalCtxt = new ExternalSPSContext(externalSps, nnc);
+      externalSps = new StoragePolicySatisfier(getConf());
+      externalCtxt = new ExternalSPSContext(externalSps, nnc);
 
 
-    externalSps.init(externalCtxt);
-    externalSps.start(StoragePolicySatisfierMode.EXTERNAL);
+      externalSps.init(externalCtxt);
+      externalSps.start(StoragePolicySatisfierMode.EXTERNAL);
+    }
     return cluster;
     return cluster;
   }
   }
 
 
@@ -1515,6 +1526,20 @@ public class TestExternalStoragePolicySatisfier {
     }
     }
   }
   }
 
 
+  @Test(timeout = 300000)
+  public void testExternalSPSMetrics()
+      throws Exception {
+
+    try {
+      createClusterDoNotStartSPS();
+      dfs.satisfyStoragePolicy(new Path(FILE));
+      // Assert metrics.
+      assertEquals(1, hdfsCluster.getNamesystem().getPendingSPSPaths());
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private static void createDirectoryTree(DistributedFileSystem dfs)
   private static void createDirectoryTree(DistributedFileSystem dfs)
       throws Exception {
       throws Exception {
     // tree structure
     // tree structure