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 um...@apache.org on 2017/08/17 20:22:56 UTC

hadoop git commit: HDFS-12214: [SPS]: Fix review comments of StoragePolicySatisfier feature. Contributed by Rakesh R.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-10285 d8122f25b -> aff40b2ba


HDFS-12214: [SPS]: Fix review comments of StoragePolicySatisfier feature. Contributed by Rakesh R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/aff40b2b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aff40b2b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aff40b2b

Branch: refs/heads/HDFS-10285
Commit: aff40b2bab6f48bcfcaad5c03ba40a0db6a6122a
Parents: d8122f2
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Thu Aug 17 13:21:07 2017 -0700
Committer: Uma Maheswara Rao G <um...@intel.com>
Committed: Thu Aug 17 13:21:07 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/bin/hdfs               |   2 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +-
 .../server/blockmanagement/BlockManager.java    | 104 +++++++++++--------
 .../BlockStorageMovementAttemptedItems.java     |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  20 ++--
 .../hadoop/hdfs/server/namenode/NameNode.java   |  22 ++--
 .../server/namenode/StoragePolicySatisfier.java |  20 ++--
 .../protocol/BlocksStorageMovementResult.java   |   2 +-
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |  11 +-
 .../src/main/resources/hdfs-default.xml         |  10 +-
 .../src/site/markdown/ArchivalStorage.md        |  14 +--
 .../src/site/markdown/HDFSCommands.md           |   2 +-
 .../TestStoragePolicySatisfyWorker.java         |   2 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |  22 ++--
 .../hdfs/server/mover/TestStorageMover.java     |   2 +-
 .../TestBlockStorageMovementAttemptedItems.java |   2 +-
 .../namenode/TestNameNodeReconfigure.java       |  99 ++++++++++++------
 .../TestPersistentStoragePolicySatisfier.java   |   6 +-
 .../namenode/TestStoragePolicySatisfier.java    |  35 +++++--
 .../TestStoragePolicySatisfierWithHA.java       |  10 +-
 ...stStoragePolicySatisfierWithStripedFile.java |   8 ++
 .../hdfs/tools/TestStoragePolicyCommands.java   |  21 ++--
 22 files changed, 265 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 594a468..a7354a8 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -59,7 +59,7 @@ function hadoop_usage
   hadoop_add_subcommand "portmap" "run a portmap service"
   hadoop_add_subcommand "secondarynamenode" "run the DFS secondary namenode"
   hadoop_add_subcommand "snapshotDiff" "diff two snapshots of a directory or diff the current directory contents with a snapshot"
-  hadoop_add_subcommand "storagepolicies" "list/get/set block storage policies"
+  hadoop_add_subcommand "storagepolicies" "list/get/set/satisfyStoragePolicy block storage policies"
   hadoop_add_subcommand "version" "print the version"
   hadoop_add_subcommand "zkfc" "run the ZK Failover Controller daemon"
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" false

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index c026d64..ec5e441 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -541,10 +541,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int    DFS_MOVER_MAX_NO_MOVE_INTERVAL_DEFAULT = 60*1000; // One minute
 
   // SPS related configurations
-  public static final String  DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY =
-      "dfs.storage.policy.satisfier.activate";
-  public static final boolean DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_DEFAULT =
-      true;
+  public static final String  DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY =
+      "dfs.storage.policy.satisfier.enabled";
+  public static final boolean DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT =
+      false;
   public static final String DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY =
       "dfs.storage.policy.satisfier.recheck.timeout.millis";
   public static final int DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 4f252a9..b55fabe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -419,6 +419,8 @@ public class BlockManager implements BlockStatsMXBean {
 
   /** For satisfying block storage policies. */
   private final StoragePolicySatisfier sps;
+  private final boolean storagePolicyEnabled;
+  private boolean spsEnabled;
   private final BlockStorageMovementNeeded storageMovementNeeded =
       new BlockStorageMovementNeeded();
 
@@ -427,15 +429,9 @@ public class BlockManager implements BlockStatsMXBean {
    */
   private final short minReplicationToBeInMaintenance;
 
-  /**
-   * Whether HA is enabled.
-   */
-  private final boolean haEnabled;
-
   public BlockManager(final Namesystem namesystem, boolean haEnabled,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
-    this.haEnabled = haEnabled;
     datanodeManager = new DatanodeManager(this, namesystem, conf);
     heartbeatManager = datanodeManager.getHeartbeatManager();
     this.blockIdManager = new BlockIdManager(this);
@@ -463,24 +459,15 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
         * 1000L);
 
-    final boolean storagePolicyEnabled =
+    storagePolicyEnabled =
         conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
-    final boolean spsEnabled =
+    spsEnabled =
         conf.getBoolean(
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_DEFAULT);
-    if (storagePolicyEnabled && spsEnabled) {
-      sps = new StoragePolicySatisfier(namesystem, storageMovementNeeded, this,
-          conf);
-    } else {
-      sps = null;
-      LOG.warn(
-          "Failed to start StoragePolicySatisfier"
-              + " since {} set to {} and {} set to {}.",
-          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled,
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, spsEnabled);
-    }
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT);
+    sps = new StoragePolicySatisfier(namesystem, storageMovementNeeded, this,
+        conf);
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     this.maxCorruptFilesReturned = conf.getInt(
@@ -700,15 +687,10 @@ public class BlockManager implements BlockStatsMXBean {
     this.blockReportThread.start();
     mxBeanName = MBeans.register("NameNode", "BlockStats", this);
     bmSafeMode.activate(blockTotal);
-    if (sps != null && !haEnabled) {
-      sps.start(false);
-    }
   }
 
   public void close() {
-    if (sps != null) {
-      sps.deactivate(false);
-    }
+    stopSPS(false);
     bmSafeMode.close();
     try {
       redundancyThread.interrupt();
@@ -4891,46 +4873,86 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Activate the storage policy satisfier by starting its service.
+   * Start storage policy satisfier service.
    */
-  public void activateSPS() {
-    if (sps == null) {
-      LOG.info("Storage policy satisfier is not initialized.");
+  public void startSPS() {
+    if (!(storagePolicyEnabled && spsEnabled)) {
+      LOG.info(
+          "Failed to start StoragePolicySatisfier "
+              + " as {} set to {} and {} set to {}.",
+          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled,
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, spsEnabled);
       return;
     } else if (sps.isRunning()) {
       LOG.info("Storage policy satisfier is already running.");
       return;
     }
 
-    sps.start(true);
+    sps.start(false);
   }
 
   /**
-   * Deactivate the storage policy satisfier by stopping its services.
+   * Stop storage policy satisfier service.
+   *
+   * @param forceStop
+   *          true represents that it should stop SPS service by clearing all
+   *          pending SPS work
    */
-  public void deactivateSPS() {
-    if (sps == null) {
-      LOG.info("Storage policy satisfier is not initialized.");
+  public void stopSPS(boolean forceStop) {
+    if (!(storagePolicyEnabled && spsEnabled)) {
+      LOG.info("Storage policy satisfier is not enabled.");
       return;
     } else if (!sps.isRunning()) {
       LOG.info("Storage policy satisfier is already stopped.");
       return;
     }
-    sps.deactivate(true);
+
+    sps.disable(forceStop);
+  }
+
+  /**
+   * Enable storage policy satisfier by starting its service.
+   */
+  public void enableSPS() {
+    if (!storagePolicyEnabled){
+      LOG.info("Failed to start StoragePolicySatisfier as {} set to {}.",
+          DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, storagePolicyEnabled);
+      return;
+    }
+    spsEnabled = true;
+    if (sps.isRunning()) {
+      LOG.info("Storage policy satisfier is already running.");
+      return;
+    }
+
+    sps.start(true);
+  }
+
+  /**
+   * Disable the storage policy satisfier by stopping its services.
+   */
+  public void disableSPS() {
+    spsEnabled = false;
+    if (!sps.isRunning()) {
+      LOG.info("Storage policy satisfier is already stopped.");
+      return;
+    }
+
+    LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
+        + "stop it.");
+    sps.disable(true);
   }
 
   /**
    * Timed wait to stop storage policy satisfier daemon threads.
    */
   public void stopSPSGracefully() {
-    if (sps != null) {
-      sps.stopGracefully();
-    }
+    sps.stopGracefully();
   }
   /**
    * @return True if storage policy satisfier running.
    */
   public boolean isStoragePolicySatisfierRunning() {
-    return sps == null ? false : sps.isRunning();
+    return sps.isRunning();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementAttemptedItems.java
index 6048986..37833e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementAttemptedItems.java
@@ -136,7 +136,7 @@ public class BlockStorageMovementAttemptedItems {
    * Sets running flag to false. Also, this will interrupt monitor thread and
    * clear all the queued up tasks.
    */
-  public synchronized void deactivate() {
+  public synchronized void stop() {
     monitorRunning = false;
     if (timerThread != null) {
       timerThread.interrupt();
@@ -152,7 +152,7 @@ public class BlockStorageMovementAttemptedItems {
       return;
     }
     if (monitorRunning) {
-      deactivate();
+      stop();
     }
     try {
       timerThread.join(3000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 3fd4f08..68931e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1267,7 +1267,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             edekCacheLoaderDelay, edekCacheLoaderInterval);
       }
 
-      blockManager.activateSPS();
+      blockManager.startSPS();
     } finally {
       startingActiveService = false;
       blockManager.checkSafeMode();
@@ -1298,7 +1298,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       if (blockManager != null) {
-        blockManager.deactivateSPS();
+        blockManager.stopSPS(true);
       }
       stopSecretManager();
       leaseManager.stopMonitor();
@@ -2141,6 +2141,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void satisfyStoragePolicy(String src, boolean logRetryCache)
       throws IOException {
+    final String operationName = "satisfyStoragePolicy";
+    FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -2158,16 +2160,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           || !blockManager.getStoragePolicySatisfier().isRunning()) {
         throw new UnsupportedActionException(
             "Cannot request to satisfy storage policy "
-                + "when storage policy satisfier feature has been deactivated"
-                + " by admin. Seek for an admin help to activate it "
+                + "when storage policy satisfier feature has been disabled"
+                + " by admin. Seek for an admin help to enable it "
                 + "or use Mover tool.");
       }
-      FSDirSatisfyStoragePolicyOp.satisfyStoragePolicy(dir, blockManager, src,
-          logRetryCache);
+      auditStat = FSDirSatisfyStoragePolicyOp.satisfyStoragePolicy(
+          dir, blockManager, src, logRetryCache);
+    } catch (AccessControlException e) {
+      logAuditEvent(false, operationName, src);
+      throw e;
     } finally {
-      writeUnlock();
+      writeUnlock(operationName);
     }
     getEditLog().logSync();
+    logAuditEvent(true, operationName, src, null, auditStat);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 672f702..a9af25b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -157,7 +157,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAUL
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.FS_PROTECTED_DIRECTORIES;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE;
@@ -290,7 +290,7 @@ public class NameNode extends ReconfigurableBase implements
           DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
           FS_PROTECTED_DIRECTORIES,
           HADOOP_CALLER_CONTEXT_ENABLED_KEY,
-          DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY));
+          DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY));
 
   private static final String USAGE = "Usage: hdfs namenode ["
       + StartupOption.BACKUP.getName() + "] | \n\t["
@@ -2036,8 +2036,8 @@ public class NameNode extends ReconfigurableBase implements
       return reconfCallerContextEnabled(newVal);
     } else if (property.equals(ipcClientRPCBackoffEnable)) {
       return reconfigureIPCBackoffEnabled(newVal);
-    } else if (property.equals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY)) {
-      return reconfigureSPSActivate(newVal, property);
+    } else if (property.equals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY)) {
+      return reconfigureSPSEnabled(newVal, property);
     } else {
       throw new ReconfigurationException(property, newVal, getConf().get(
           property));
@@ -2118,29 +2118,29 @@ public class NameNode extends ReconfigurableBase implements
     return Boolean.toString(clientBackoffEnabled);
   }
 
-  String reconfigureSPSActivate(String newVal, String property)
+  String reconfigureSPSEnabled(String newVal, String property)
       throws ReconfigurationException {
     if (newVal == null || !(newVal.equalsIgnoreCase(Boolean.TRUE.toString())
         || newVal.equalsIgnoreCase(Boolean.FALSE.toString()))) {
       throw new ReconfigurationException(property, newVal,
           getConf().get(property),
           new HadoopIllegalArgumentException(
-              "For activating or deactivating storage policy satisfier, "
+              "For enabling or disabling storage policy satisfier, "
                   + "we must pass true/false only"));
     }
 
     if (!isActiveState()) {
       throw new ReconfigurationException(property, newVal,
           getConf().get(property), new HadoopIllegalArgumentException(
-          "Activating or deactivating storage policy satisfier service on "
+          "Enabling or disabling storage policy satisfier service on "
               + state + " NameNode is not allowed"));
     }
 
-    boolean activateSPS = Boolean.parseBoolean(newVal);
-    if (activateSPS) {
-      namesystem.getBlockManager().activateSPS();
+    boolean enableSPS = Boolean.parseBoolean(newVal);
+    if (enableSPS) {
+      namesystem.getBlockManager().enableSPS();
     } else {
-      namesystem.getBlockManager().deactivateSPS();
+      namesystem.getBlockManager().disableSPS();
     }
     return newVal;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
index af3b7f2..3165813 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
@@ -138,7 +138,7 @@ public class StoragePolicySatisfier implements Runnable {
     }
     if (reconfigStart) {
       LOG.info("Starting StoragePolicySatisfier, as admin requested to "
-          + "activate it.");
+          + "start it.");
     } else {
       LOG.info("Starting StoragePolicySatisfier.");
     }
@@ -154,23 +154,21 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   /**
-   * Deactivates storage policy satisfier by stopping its services.
+   * Disables storage policy satisfier by stopping its services.
    *
-   * @param reconfig
-   *          true represents deactivating SPS service as requested by admin,
-   *          false otherwise
+   * @param forceStop
+   *          true represents that it should stop SPS service by clearing all
+   *          pending SPS work
    */
-  public synchronized void deactivate(boolean reconfig) {
+  public synchronized void disable(boolean forceStop) {
     isRunning = false;
     if (storagePolicySatisfierThread == null) {
       return;
     }
 
     storagePolicySatisfierThread.interrupt();
-    this.storageMovementsMonitor.deactivate();
-    if (reconfig) {
-      LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
-          + "deactivate it.");
+    this.storageMovementsMonitor.stop();
+    if (forceStop) {
       this.clearQueuesWithNotification();
       addDropSPSWorkCommandsToAllDNs();
     } else {
@@ -183,7 +181,7 @@ public class StoragePolicySatisfier implements Runnable {
    */
   public synchronized void stopGracefully() {
     if (isRunning) {
-      deactivate(true);
+      disable(true);
     }
     this.storageMovementsMonitor.stopGracefully();
     if (storagePolicySatisfierThread == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMovementResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMovementResult.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMovementResult.java
index b484eb1..7f749ec4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMovementResult.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMovementResult.java
@@ -40,7 +40,7 @@ public class BlocksStorageMovementResult {
    * IN_PROGRESS - If all or some of the blocks associated to track id are
    * still moving.
    */
-  public static enum Status {
+  public enum Status {
     SUCCESS, FAILURE, IN_PROGRESS;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
index 30420ad..9b57874 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
@@ -233,8 +233,8 @@ public class StoragePolicyAdmin extends Configured implements Tool {
   }
 
   /** Command to schedule blocks to move based on specified policy. */
-  private static class SatisfyStoragePolicyCommand implements
-      AdminHelper.Command {
+  private static class SatisfyStoragePolicyCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
       return "-satisfyStoragePolicy";
@@ -278,10 +278,11 @@ public class StoragePolicyAdmin extends Configured implements Tool {
   }
 
   /** Command to check storage policy satisfier status. */
-  private static class IsSPSRunningCommand implements AdminHelper.Command {
+  private static class IsSatisfierRunningCommand
+      implements AdminHelper.Command {
     @Override
     public String getName() {
-      return "-isSPSRunning";
+      return "-isSatisfierRunning";
     }
 
     @Override
@@ -370,6 +371,6 @@ public class StoragePolicyAdmin extends Configured implements Tool {
       new GetStoragePolicyCommand(),
       new UnsetStoragePolicyCommand(),
       new SatisfyStoragePolicyCommand(),
-      new IsSPSRunningCommand()
+      new IsSatisfierRunningCommand()
   };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 2b1e28a..58ebc18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4213,13 +4213,13 @@
   </property>
 
 <property>
-  <name>dfs.storage.policy.satisfier.activate</name>
-  <value>true</value>
+  <name>dfs.storage.policy.satisfier.enabled</name>
+  <value>false</value>
   <description>
     If true, StoragePolicySatisfier will be started along with active namenode.
-    By default, StoragePolicySatisfier is activated.
-    Administrator can dynamically activate or deactivate StoragePolicySatisfier by using reconfiguration option.
-    Dynamic activation/deactivation option can be achieved in the following way.
+    By default, StoragePolicySatisfier is disabled.
+    Administrator can dynamically enable or disable StoragePolicySatisfier by using reconfiguration option.
+    Dynamic enabling/disabling option can be achieved in the following way.
     1. Edit/update this configuration property values in hdfs-site.xml
     2. Execute the reconfig command on hadoop command line prompt.
        For example:$hdfs -reconfig namenode nn_host:port start

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
index 668bb20..87817cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -106,7 +106,7 @@ Following 2 options will allow users to move the blocks based on new policy set.
 When user changes the storage policy on a file/directory, user can call `HdfsAdmin` API `satisfyStoragePolicy()` to move the blocks as per the new policy set.
 The SPS daemon thread runs along with namenode and periodically scans for the storage mismatches between new policy set and the physical blocks placed. This will only track the files/directories for which user invoked satisfyStoragePolicy. If SPS identifies some blocks to be moved for a file, then it will schedule block movement tasks to datanodes. A Coordinator DataNode(C-DN) will track all block movements associated to a file and notify to namenode about movement success/failure. If there are any failures in movement, the SPS will re-attempt by sending new block movement task.
 
-SPS can be activated and deactivated dynamically without restarting the Namenode.
+SPS can be enabled and disabled dynamically without restarting the Namenode.
 
 Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HDFS-10285)](https://issues.apache.org/jira/browse/HDFS-10285)
 
@@ -123,8 +123,8 @@ Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HD
 
 ####Configurations:
 
-*   **dfs.storage.policy.satisfier.activate** - Used to activate or deactivate SPS. Configuring true represents SPS is
-   activated and vice versa.
+*   **dfs.storage.policy.satisfier.enabled** - Used to enable or disable SPS. Configuring true represents SPS is
+   enabled and vice versa.
 
 *   **dfs.storage.policy.satisfier.recheck.timeout.millis** - A timeout to re-check the processed block storage movement
    command results from Co-ordinator Datanode.
@@ -151,7 +151,7 @@ Note that, when both -p and -f options are omitted, the default path is the root
 
 ####Administrator notes:
 
-`StoragePolicySatisfier` and `Mover tool` cannot run simultaneously. If a Mover instance is already triggered and running, SPS will be deactivated while starting. In that case, administrator should make sure, Mover execution finished and then activate SPS again. Similarly when SPS activated already, Mover cannot be run. If administrator is looking to run Mover tool explicitly, then he/she should make sure to deactivate SPS first and then run Mover. Please look at the commands section to know how to activate or deactivate SPS dynamically.
+`StoragePolicySatisfier` and `Mover tool` cannot run simultaneously. If a Mover instance is already triggered and running, SPS will be disabled while starting. In that case, administrator should make sure, Mover execution finished and then enable SPS again. Similarly when SPS enabled already, Mover cannot be run. If administrator is looking to run Mover tool explicitly, then he/she should make sure to disable SPS first and then run Mover. Please look at the commands section to know how to enable or disable SPS dynamically.
 
 Storage Policy Commands
 -----------------------
@@ -230,10 +230,10 @@ Check the running status of Storage Policy Satisfier in namenode. If it is runni
 
 * Command:
 
-        hdfs storagepolicies -isSPSRunning
+        hdfs storagepolicies -isSatisfierRunning
 
-### Activate or Deactivate SPS without restarting Namenode
-If administrator wants to activate or deactivate SPS feature while Namenode is running, first he/she needs to update the desired value(true or false) for the configuration item `dfs.storage.policy.satisfier.activate` in configuration file (`hdfs-site.xml`) and then run the following Namenode reconfig command
+### Enable or Disable SPS without restarting Namenode
+If administrator wants to enable or disable SPS feature while Namenode is running, first he/she needs to update the desired value(true or false) for the configuration item `dfs.storage.policy.satisfier.enabled` in configuration file (`hdfs-site.xml`) and then run the following Namenode reconfig command
 
 +       hdfs dfsadmin -reconfig namenode <host:ipc_port> start
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index f5ad187..29cdf37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -577,7 +577,7 @@ Usage:
           [-getStoragePolicy -path <path>]
           [-unsetStoragePolicy -path <path>]
           [-satisfyStoragePolicy -path <path>]
-          [-isSPSRunning]
+          [-isSatisfierRunning]
           [-help <command-name>]
 
 Lists out all/Gets/sets/unsets storage policies. See the [HDFS Storage Policy Documentation](./ArchivalStorage.html) for more information.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
index 8fbbf33..402d4d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
@@ -69,7 +69,7 @@ public class TestStoragePolicySatisfyWorker {
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
         1L);
     conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index c4f9bfa..2b9c186 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -114,7 +114,7 @@ public class TestMover {
         1L);
     conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
   }
 
   static Mover newMover(Configuration conf) throws IOException {
@@ -136,7 +136,7 @@ public class TestMover {
   public void testScheduleSameBlock() throws IOException {
     final Configuration conf = new HdfsConfiguration();
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(4).build();
     try {
@@ -253,7 +253,7 @@ public class TestMover {
   public void testMoverCli() throws Exception {
     final Configuration clusterConf = new HdfsConfiguration();
     clusterConf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(clusterConf).numDataNodes(0).build();
     try {
@@ -288,7 +288,7 @@ public class TestMover {
   public void testMoverCliWithHAConf() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
@@ -313,14 +313,14 @@ public class TestMover {
   public void testMoverCliWithFederation() throws Exception {
     final Configuration clusterConf = new HdfsConfiguration();
     clusterConf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(clusterConf)
         .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
         .numDataNodes(0).build();
     final Configuration conf = new HdfsConfiguration();
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     DFSTestUtil.setFederatedConfiguration(cluster, conf);
     try {
       Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -366,14 +366,14 @@ public class TestMover {
   public void testMoverCliWithFederationHA() throws Exception {
     final Configuration clusterConf = new HdfsConfiguration();
     clusterConf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(clusterConf)
         .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(3))
         .numDataNodes(0).build();
     final Configuration conf = new HdfsConfiguration();
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     DFSTestUtil.setFederatedHAConfiguration(cluster, conf);
     try {
       Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -438,7 +438,7 @@ public class TestMover {
     // HDFS-8147
     final Configuration conf = new HdfsConfiguration();
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(3)
         .storageTypes(
@@ -468,7 +468,7 @@ public class TestMover {
   public void testMoveWhenStoragePolicySatisfierIsRunning() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, true);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(3)
         .storageTypes(
@@ -545,7 +545,7 @@ public class TestMover {
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
     conf.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
   }
 
   @Test(timeout = 300000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index 7e42fbe..3ddb37f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -97,7 +97,7 @@ public class TestStorageMover {
         DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 2L);
     DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L);
     DEFAULT_CONF.setBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
 
     DEFAULT_POLICIES = BlockStoragePolicySuite.createDefaultSuite();
     HOT = DEFAULT_POLICIES.getPolicy(HdfsConstants.HOT_STORAGE_POLICY_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockStorageMovementAttemptedItems.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockStorageMovementAttemptedItems.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockStorageMovementAttemptedItems.java
index 8c7d982..04a63ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockStorageMovementAttemptedItems.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockStorageMovementAttemptedItems.java
@@ -47,7 +47,7 @@ public class TestBlockStorageMovementAttemptedItems {
   @After
   public void teardown() {
     if (bsmAttemptedItems != null) {
-      bsmAttemptedItems.deactivate();
+      bsmAttemptedItems.stop();
       bsmAttemptedItems.stopGracefully();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java
index a3188a3..8c2bd2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -44,8 +45,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE_DEFAULT;
 
 public class TestNameNodeReconfigure {
@@ -208,63 +209,99 @@ public class TestNameNodeReconfigure {
   }
 
   /**
-   * Tests activate/deactivate Storage Policy Satisfier dynamically.
+   * Tests enable/disable Storage Policy Satisfier dynamically when
+   * "dfs.storage.policy.enabled" feature is disabled.
+   *
+   * @throws ReconfigurationException
+   * @throws IOException
    */
   @Test(timeout = 30000)
-  public void testReconfigureStoragePolicySatisfierActivated()
+  public void testReconfigureSPSWithStoragePolicyDisabled()
+      throws ReconfigurationException, IOException {
+    // shutdown cluster
+    cluster.shutdown();
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
+    cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster.waitActive();
+
+    final NameNode nameNode = cluster.getNameNode();
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false);
+
+    // enable SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        "true");
+
+    // Since DFS_STORAGE_POLICY_ENABLED_KEY is disabled, SPS can't be enabled.
+    assertEquals("SPS shouldn't start as "
+        + DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY + " is disabled", false,
+            nameNode.getNamesystem().getBlockManager()
+            .isStoragePolicySatisfierRunning());
+
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
+        true, nameNode.getConf()
+            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+            DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT));
+  }
+
+  /**
+   * Tests enable/disable Storage Policy Satisfier dynamically.
+   */
+  @Test(timeout = 30000)
+  public void testReconfigureStoragePolicySatisfierEnabled()
       throws ReconfigurationException {
     final NameNode nameNode = cluster.getNameNode();
 
-    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
-        true);
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        false);
     // try invalid values
     try {
-      nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+      nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
           "text");
       fail("ReconfigurationException expected");
     } catch (ReconfigurationException e) {
       GenericTestUtils.assertExceptionContains(
-          "For activating or deactivating storage policy satisfier, "
+          "For enabling or disabling storage policy satisfier, "
               + "we must pass true/false only",
           e.getCause());
     }
 
     // enable SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         "true");
 
-    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         true);
 
     // disable SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         "false");
-    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         false);
 
-    // revert to default
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    // enable SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         "true");
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
         true, nameNode.getNamesystem().getBlockManager()
             .isStoragePolicySatisfierRunning());
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
         true, nameNode.getConf()
-            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false));
+            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false));
   }
 
   /**
-   * Test to satisfy storage policy after deactivating storage policy satisfier.
+   * Test to satisfy storage policy after disabled storage policy satisfier.
    */
   @Test(timeout = 30000)
-  public void testSatisfyStoragePolicyAfterSatisfierDeactivated()
+  public void testSatisfyStoragePolicyAfterSatisfierDisabled()
       throws ReconfigurationException, IOException {
     final NameNode nameNode = cluster.getNameNode();
 
-    // deactivate SPS
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    // disable SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         "false");
-    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         false);
 
     Path filePath = new Path("/testSPS");
@@ -273,32 +310,32 @@ public class TestNameNodeReconfigure {
     fileSystem.setStoragePolicy(filePath, "COLD");
     try {
       fileSystem.satisfyStoragePolicy(filePath);
-      fail("Expected to fail, as storage policy feature has deactivated.");
+      fail("Expected to fail, as storage policy feature has disabled.");
     } catch (RemoteException e) {
       GenericTestUtils
           .assertExceptionContains("Cannot request to satisfy storage policy "
-              + "when storage policy satisfier feature has been deactivated"
-              + " by admin. Seek for an admin help to activate it "
+              + "when storage policy satisfier feature has been disabled"
+              + " by admin. Seek for an admin help to enable it "
               + "or use Mover tool.", e);
     }
 
     // revert to default
-    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
         "true");
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
         true, nameNode.getNamesystem().getBlockManager()
             .isStoragePolicySatisfierRunning());
-    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY + " has wrong value",
         true, nameNode.getConf()
-            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false));
+            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, false));
   }
 
-  void verifySPSActivated(final NameNode nameNode, String property,
+  void verifySPSEnabled(final NameNode nameNode, String property,
       boolean expected) {
     assertEquals(property + " has wrong value", expected, nameNode
         .getNamesystem().getBlockManager().isStoragePolicySatisfierRunning());
     assertEquals(property + " has wrong value", expected, nameNode.getConf()
-        .getBoolean(property, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_DEFAULT));
+        .getBoolean(property, DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT));
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPersistentStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPersistentStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPersistentStoragePolicySatisfier.java
index bdf0159..8516ea0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPersistentStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPersistentStoragePolicySatisfier.java
@@ -96,6 +96,8 @@ public class TestPersistentStoragePolicySatisfier {
     conf.set(
         DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         "3000");
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     final int dnNumber = storageTypes.length;
     final short replication = 3;
     MiniDFSCluster.Builder clusterBuilder = new MiniDFSCluster.Builder(conf)
@@ -282,6 +284,8 @@ public class TestPersistentStoragePolicySatisfier {
     MiniDFSCluster haCluster = null;
     try {
       conf = new HdfsConfiguration();
+      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
       haCluster = new MiniDFSCluster
           .Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
@@ -376,7 +380,7 @@ public class TestPersistentStoragePolicySatisfier {
       fs.setStoragePolicy(testFile, ONE_SSD);
       fs.satisfyStoragePolicy(testFile);
 
-      cluster.getNamesystem().getBlockManager().deactivateSPS();
+      cluster.getNamesystem().getBlockManager().disableSPS();
 
       // Make sure satisfy xattr has been removed.
       DFSTestUtil.waitForXattrRemoved(testFileName,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
index 7f96003..2536834 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
@@ -96,6 +96,8 @@ public class TestStoragePolicySatisfier {
 
   private void createCluster() throws IOException {
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     hdfsCluster = startCluster(config, allDiskTypes, numOfDatanodes,
         storagesPerDatanode, capacity);
     dfs = hdfsCluster.getFileSystem();
@@ -522,7 +524,7 @@ public class TestStoragePolicySatisfier {
       createCluster();
       // Stop SPS
       hdfsCluster.getNameNode().reconfigurePropertyImpl(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "false");
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
       running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
       Assert.assertFalse("SPS should stopped as configured.", running);
@@ -533,7 +535,7 @@ public class TestStoragePolicySatisfier {
 
       // Restart SPS
       hdfsCluster.getNameNode().reconfigurePropertyImpl(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "true");
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "true");
 
       running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
@@ -548,7 +550,7 @@ public class TestStoragePolicySatisfier {
 
       // Restart SPS again
       hdfsCluster.getNameNode().reconfigurePropertyImpl(
-          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "true");
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "true");
       running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
       Assert.assertTrue("SPS should be running as "
@@ -558,7 +560,7 @@ public class TestStoragePolicySatisfier {
       doTestWhenStoragePolicySetToCOLD();
     } catch (ReconfigurationException e) {
       throw new IOException("Exception when reconfigure "
-          + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, e);
+          + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, e);
     } finally {
       if (out != null) {
         out.close();
@@ -599,6 +601,8 @@ public class TestStoragePolicySatisfier {
   @Test(timeout = 120000)
   public void testMoveWithBlockPinning() throws Exception {
     config.setBoolean(DFSConfigKeys.DFS_DATANODE_BLOCK_PINNING_ENABLED, true);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(3)
         .storageTypes(
             new StorageType[][] {{StorageType.DISK, StorageType.DISK},
@@ -663,6 +667,8 @@ public class TestStoragePolicySatisfier {
     try {
       int numOfDns = 5;
       config.setLong("dfs.block.size", 1024);
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
       allDiskTypes =
           new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
               {StorageType.DISK, StorageType.DISK},
@@ -707,6 +713,8 @@ public class TestStoragePolicySatisfier {
             {StorageType.DISK, StorageType.SSD},
             {StorageType.DISK, StorageType.RAM_DISK}};
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     try {
       hdfsCluster = startCluster(config, diskTypes, numOfDatanodes,
           storagesPerDatanode, capacity);
@@ -746,6 +754,8 @@ public class TestStoragePolicySatisfier {
             {StorageType.DISK, StorageType.DISK}};
 
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     try {
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
           storagesPerDatanode, capacity);
@@ -782,6 +792,8 @@ public class TestStoragePolicySatisfier {
         {StorageType.DISK, StorageType.ARCHIVE}};
 
     try {
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
           storagesPerDatanode, capacity);
       dfs = hdfsCluster.getFileSystem();
@@ -825,6 +837,8 @@ public class TestStoragePolicySatisfier {
             {StorageType.DISK, StorageType.SSD},
             {StorageType.DISK, StorageType.DISK}};
     config.setLong("dfs.block.size", 2 * DEFAULT_BLOCK_SIZE);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     long dnCapacity = 1024 * DEFAULT_BLOCK_SIZE + (2 * DEFAULT_BLOCK_SIZE - 1);
     try {
       hdfsCluster = startCluster(config, diskTypes, numOfDatanodes,
@@ -915,7 +929,8 @@ public class TestStoragePolicySatisfier {
         1L);
     config.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
-
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     try {
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
           storagesPerDatanode, capacity);
@@ -968,8 +983,10 @@ public class TestStoragePolicySatisfier {
   public void testSPSWhenFileLengthIsZero() throws Exception {
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(0)
-          .build();
+      Configuration conf = new Configuration();
+      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
       Path filePath = new Path("/zeroSizeFile");
@@ -1006,6 +1023,8 @@ public class TestStoragePolicySatisfier {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
+      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
       conf.set(DFSConfigKeys
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");
@@ -1054,6 +1073,8 @@ public class TestStoragePolicySatisfier {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
+      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
       conf.set(DFSConfigKeys
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithHA.java
index c88d5be..b0fd3af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithHA.java
@@ -65,6 +65,8 @@ public class TestStoragePolicySatisfierWithHA {
 
   private void createCluster() throws IOException {
     config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     startCluster(config, allDiskTypes, numOfDatanodes, storagesPerDatanode,
         capacity);
     dfs = cluster.getFileSystem(nnIndex);
@@ -131,15 +133,15 @@ public class TestStoragePolicySatisfierWithHA {
 
       try {
         cluster.getNameNode(0).reconfigurePropertyImpl(
-            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "false");
-        Assert.fail("It's not allowed to activate or deactivate"
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
+        Assert.fail("It's not allowed to enable or disable"
             + " StoragePolicySatisfier on Standby NameNode");
       } catch (ReconfigurationException e) {
         GenericTestUtils.assertExceptionContains("Could not change property "
-            + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY
+            + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY
             + " from 'true' to 'false'", e);
         GenericTestUtils.assertExceptionContains(
-            "Activating or deactivating storage policy satisfier service on "
+            "Enabling or disabling storage policy satisfier service on "
                 + "standby NameNode is not allowed", e.getCause());
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithStripedFile.java
index c070113..fc5d0a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithStripedFile.java
@@ -103,6 +103,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }
 
     final Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
         StripedFileTestUtil.getDefaultECPolicy().getName());
     initConfWithStripe(conf, defaultStripeBlockSize);
@@ -215,6 +217,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     final Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
         StripedFileTestUtil.getDefaultECPolicy().getName());
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -325,6 +329,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
         "3000");
     conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
         StripedFileTestUtil.getDefaultECPolicy().getName());
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -415,6 +421,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     final Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
         StripedFileTestUtil.getDefaultECPolicy().getName());
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aff40b2b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
index c86eecd..e6a0356 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
@@ -47,6 +47,8 @@ public class TestStoragePolicyCommands {
   @Before
   public void clusterSetUp() throws IOException {
     conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+        true);
     StorageType[][] newtypes = new StorageType[][] {
         {StorageType.ARCHIVE, StorageType.DISK}};
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL)
@@ -163,7 +165,7 @@ public class TestStoragePolicyCommands {
         "File/Directory does not exist: /fooz");
   }
 
-  @Test
+  @Test(timeout = 30000)
   public void testStoragePolicySatisfierCommand() throws Exception {
     final String file = "/testStoragePolicySatisfierCommand";
     DFSTestUtil.createFile(fs, new Path(file), SIZE, REPL, 0);
@@ -184,18 +186,21 @@ public class TestStoragePolicyCommands {
         fs);
   }
 
-  @Test
-  public void testIsSPSRunningCommand() throws Exception {
-    final String file = "/testIsSPSRunningCommand";
+  @Test(timeout = 30000)
+  public void testIsSatisfierRunningCommand() throws Exception {
+    final String file = "/testIsSatisfierRunningCommand";
     DFSTestUtil.createFile(fs, new Path(file), SIZE, REPL, 0);
     final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
-    DFSTestUtil.toolRun(admin, "-isSPSRunning", 0, "yes");
+    DFSTestUtil.toolRun(admin, "-isSatisfierRunning", 0, "yes");
+
     cluster.getNameNode().reconfigureProperty(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "false");
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
     cluster.waitActive();
-    DFSTestUtil.toolRun(admin, "-isSPSRunning", 0, "no");
+
+    DFSTestUtil.toolRun(admin, "-isSatisfierRunning", 0, "no");
+
     // Test with unnecessary args
-    DFSTestUtil.toolRun(admin, "-isSPSRunning status", 1,
+    DFSTestUtil.toolRun(admin, "-isSatisfierRunning status", 1,
         "Can't understand arguments: ");
   }
 }


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