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 ra...@apache.org on 2017/12/11 07:15:49 UTC

[01/50] [abbrv] hadoop git commit: YARN-6704. Add support for work preserving NM restart when FederationInterceptor is enabled in AMRMProxyService. (Botong Huang via Subru). [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-10285 04ed6dd1f -> 73f7db619 (forced update)


YARN-6704. Add support for work preserving NM restart when FederationInterceptor is enabled in AMRMProxyService. (Botong Huang via Subru).


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

Branch: refs/heads/HDFS-10285
Commit: 670e8d4ec7e71fc3b054cd3b2826f869b649a788
Parents: 04b84da
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Dec 8 15:39:18 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Fri Dec 8 15:39:18 2017 -0800

----------------------------------------------------------------------
 .../yarn/server/MockResourceManagerFacade.java  |  16 +-
 .../nodemanager/amrmproxy/AMRMProxyService.java |   5 +-
 .../amrmproxy/FederationInterceptor.java        | 271 +++++++++++++++++--
 .../amrmproxy/BaseAMRMProxyTest.java            |  15 +
 .../amrmproxy/TestFederationInterceptor.java    | 104 +++++++
 .../TestableFederationInterceptor.java          |   8 +-
 6 files changed, 387 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index b5727aa..15e1cea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -111,6 +111,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@@ -622,7 +623,20 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
     validateRunning();
 
-    return GetContainersResponse.newInstance(null);
+    ApplicationAttemptId attemptId = request.getApplicationAttemptId();
+    List<ContainerReport> containers = new ArrayList<>();
+    synchronized (applicationContainerIdMap) {
+      // Return the list of running containers that were being tracked for this
+      // application
+      Assert.assertTrue("The application id is NOT registered: " + attemptId,
+          applicationContainerIdMap.containsKey(attemptId));
+      List<ContainerId> ids = applicationContainerIdMap.get(attemptId);
+      for (ContainerId c : ids) {
+        containers.add(ContainerReport.newInstance(c, null, null, null, 0, 0,
+            null, null, 0, null, null));
+      }
+    }
+    return GetContainersResponse.newInstance(containers);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
index ebd85bf..815e39b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
@@ -128,11 +128,8 @@ public class AMRMProxyService extends CompositeService implements
         new AMRMProxyTokenSecretManager(this.nmContext.getNMStateStore());
     this.secretManager.init(conf);
 
-    // Both second app attempt and NM restart within Federation need registry
     if (conf.getBoolean(YarnConfiguration.AMRM_PROXY_HA_ENABLED,
-        YarnConfiguration.DEFAULT_AMRM_PROXY_HA_ENABLED)
-        || conf.getBoolean(YarnConfiguration.NM_RECOVERY_ENABLED,
-            YarnConfiguration.DEFAULT_NM_RECOVERY_ENABLED)) {
+        YarnConfiguration.DEFAULT_AMRM_PROXY_HA_ENABLED)) {
       this.registry = FederationStateStoreFacade.createInstance(conf,
           YarnConfiguration.YARN_REGISTRY_CLASS,
           YarnConfiguration.DEFAULT_YARN_REGISTRY_CLASS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
index ef5e061..9a53a50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -37,16 +37,23 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.PreemptionContract;
@@ -59,6 +66,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProto;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
@@ -90,6 +99,22 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private static final Logger LOG =
       LoggerFactory.getLogger(FederationInterceptor.class);
 
+  public static final String NMSS_CLASS_PREFIX = "FederationInterceptor/";
+
+  public static final String NMSS_REG_REQUEST_KEY =
+      NMSS_CLASS_PREFIX + "registerRequest";
+  public static final String NMSS_REG_RESPONSE_KEY =
+      NMSS_CLASS_PREFIX + "registerResponse";
+
+  /*
+   * When AMRMProxy HA is enabled, secondary AMRMTokens will be stored in Yarn
+   * Registry. Otherwise if NM recovery is enabled, the UAM token are store in
+   * local NMSS instead under this directory name.
+   */
+  public static final String NMSS_SECONDARY_SC_PREFIX =
+      NMSS_CLASS_PREFIX + "secondarySC/";
+  public static final String STRING_TO_BYTE_FORMAT = "UTF-8";
+
   /**
    * The home sub-cluster is the sub-cluster where the AM container is running
    * in.
@@ -187,14 +212,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     } catch (Exception ex) {
       throw new YarnRuntimeException(ex);
     }
-    // Add all app tokens for Yarn Registry access
-    if (this.registryClient != null && appContext.getCredentials() != null) {
-      this.appOwner.addCredentials(appContext.getCredentials());
+
+    if (appContext.getRegistryClient() != null) {
+      this.registryClient = new FederationRegistryClient(conf,
+          appContext.getRegistryClient(), this.appOwner);
+      // Add all app tokens for Yarn Registry access
+      if (appContext.getCredentials() != null) {
+        this.appOwner.addCredentials(appContext.getCredentials());
+      }
     }
 
     this.homeSubClusterId =
         SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
-    this.homeRM = createHomeRMProxy(appContext);
+    this.homeRM = createHomeRMProxy(appContext, ApplicationMasterProtocol.class,
+        this.appOwner);
 
     this.federationFacade = FederationStateStoreFacade.getInstance();
     this.subClusterResolver = this.federationFacade.getSubClusterResolver();
@@ -204,11 +235,137 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
     this.uamPool.init(conf);
     this.uamPool.start();
+  }
 
-    if (appContext.getRegistryClient() != null) {
-      this.registryClient = new FederationRegistryClient(conf,
-          appContext.getRegistryClient(), this.appOwner);
+  @Override
+  public void recover(Map<String, byte[]> recoveredDataMap) {
+    super.recover(recoveredDataMap);
+    LOG.info("Recovering data for FederationInterceptor");
+    if (recoveredDataMap == null) {
+      return;
     }
+
+    ApplicationAttemptId attemptId =
+        getApplicationContext().getApplicationAttemptId();
+    try {
+      if (recoveredDataMap.containsKey(NMSS_REG_REQUEST_KEY)) {
+        RegisterApplicationMasterRequestProto pb =
+            RegisterApplicationMasterRequestProto
+                .parseFrom(recoveredDataMap.get(NMSS_REG_REQUEST_KEY));
+        this.amRegistrationRequest =
+            new RegisterApplicationMasterRequestPBImpl(pb);
+        LOG.info("amRegistrationRequest recovered for {}", attemptId);
+      }
+      if (recoveredDataMap.containsKey(NMSS_REG_RESPONSE_KEY)) {
+        RegisterApplicationMasterResponseProto pb =
+            RegisterApplicationMasterResponseProto
+                .parseFrom(recoveredDataMap.get(NMSS_REG_RESPONSE_KEY));
+        this.amRegistrationResponse =
+            new RegisterApplicationMasterResponsePBImpl(pb);
+        LOG.info("amRegistrationResponse recovered for {}", attemptId);
+      }
+
+      // Recover UAM amrmTokens from registry or NMSS
+      Map<String, Token<AMRMTokenIdentifier>> uamMap;
+      if (this.registryClient != null) {
+        uamMap = this.registryClient
+            .loadStateFromRegistry(attemptId.getApplicationId());
+        LOG.info("Found {} existing UAMs for application {} in Yarn Registry",
+            uamMap.size(), attemptId.getApplicationId());
+      } else {
+        uamMap = new HashMap<>();
+        for (Entry<String, byte[]> entry : recoveredDataMap.entrySet()) {
+          if (entry.getKey().startsWith(NMSS_SECONDARY_SC_PREFIX)) {
+            // entry for subClusterId -> UAM amrmToken
+            String scId =
+                entry.getKey().substring(NMSS_SECONDARY_SC_PREFIX.length());
+            Token<AMRMTokenIdentifier> amrmToken = new Token<>();
+            amrmToken.decodeFromUrlString(
+                new String(entry.getValue(), STRING_TO_BYTE_FORMAT));
+            uamMap.put(scId, amrmToken);
+            LOG.debug("Recovered UAM in " + scId + " from NMSS");
+          }
+        }
+        LOG.info("Found {} existing UAMs for application {} in NMStateStore",
+            uamMap.size(), attemptId.getApplicationId());
+      }
+
+      // Re-attach the UAMs
+      int containers = 0;
+      for (Map.Entry<String, Token<AMRMTokenIdentifier>> entry : uamMap
+          .entrySet()) {
+        SubClusterId subClusterId = SubClusterId.newInstance(entry.getKey());
+
+        // Create a config loaded with federation on and subclusterId
+        // for each UAM
+        YarnConfiguration config = new YarnConfiguration(getConf());
+        FederationProxyProviderUtil.updateConfForFederation(config,
+            subClusterId.getId());
+
+        try {
+          this.uamPool.reAttachUAM(subClusterId.getId(), config,
+              attemptId.getApplicationId(),
+              this.amRegistrationResponse.getQueue(),
+              getApplicationContext().getUser(), this.homeSubClusterId.getId(),
+              entry.getValue());
+
+          RegisterApplicationMasterResponse response =
+              this.uamPool.registerApplicationMaster(subClusterId.getId(),
+                  this.amRegistrationRequest);
+
+          // Running containers from secondary RMs
+          for (Container container : response
+              .getContainersFromPreviousAttempts()) {
+            containerIdToSubClusterIdMap.put(container.getId(), subClusterId);
+            containers++;
+          }
+          LOG.info("Recovered {} running containers from UAM in {}",
+              response.getContainersFromPreviousAttempts().size(),
+              subClusterId);
+
+        } catch (Exception e) {
+          LOG.error(
+              "Error reattaching UAM to " + subClusterId + " for " + attemptId,
+              e);
+        }
+      }
+
+      // Get the running containers from home RM, note that we will also get the
+      // AM container itself from here. We don't need it, but no harm to put the
+      // map as well.
+      UserGroupInformation appSubmitter = UserGroupInformation
+          .createRemoteUser(getApplicationContext().getUser());
+      ApplicationClientProtocol rmClient =
+          createHomeRMProxy(getApplicationContext(),
+              ApplicationClientProtocol.class, appSubmitter);
+
+      GetContainersResponse response =
+          rmClient.getContainers(GetContainersRequest.newInstance(attemptId));
+      for (ContainerReport container : response.getContainerList()) {
+        containerIdToSubClusterIdMap.put(container.getContainerId(),
+            this.homeSubClusterId);
+        containers++;
+        LOG.debug("  From home RM " + this.homeSubClusterId
+            + " running container " + container.getContainerId());
+      }
+      LOG.info("{} running containers including AM recovered from home RM ",
+          response.getContainerList().size(), this.homeSubClusterId);
+
+      LOG.info(
+          "In all {} UAMs {} running containers including AM recovered for {}",
+          uamMap.size(), containers, attemptId);
+
+      if (this.amRegistrationResponse != null) {
+        // Initialize the AMRMProxyPolicy
+        String queue = this.amRegistrationResponse.getQueue();
+        this.policyInterpreter =
+            FederationPolicyUtils.loadAMRMPolicy(queue, this.policyInterpreter,
+                getConf(), this.federationFacade, this.homeSubClusterId);
+      }
+    } catch (IOException | YarnException e) {
+      throw new YarnRuntimeException(e);
+    }
+
   }
 
   /**
@@ -242,6 +399,19 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       // Save the registration request. This will be used for registering with
       // secondary sub-clusters using UAMs, as well as re-register later
       this.amRegistrationRequest = request;
+      if (getNMStateStore() != null) {
+        try {
+          RegisterApplicationMasterRequestPBImpl pb =
+              (RegisterApplicationMasterRequestPBImpl)
+                  this.amRegistrationRequest;
+          getNMStateStore().storeAMRMProxyAppContextEntry(
+              getApplicationContext().getApplicationAttemptId(),
+              NMSS_REG_REQUEST_KEY, pb.getProto().toByteArray());
+        } catch (Exception e) {
+          LOG.error("Error storing AMRMProxy application context entry for "
+              + getApplicationContext().getApplicationAttemptId(), e);
+        }
+      }
     }
 
     /*
@@ -278,6 +448,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
         getApplicationContext().getApplicationAttemptId().getApplicationId();
     reAttachUAMAndMergeRegisterResponse(this.amRegistrationResponse, appId);
 
+    if (getNMStateStore() != null) {
+      try {
+        RegisterApplicationMasterResponsePBImpl pb =
+            (RegisterApplicationMasterResponsePBImpl)
+                this.amRegistrationResponse;
+        getNMStateStore().storeAMRMProxyAppContextEntry(
+            getApplicationContext().getApplicationAttemptId(),
+            NMSS_REG_RESPONSE_KEY, pb.getProto().toByteArray());
+      } catch (Exception e) {
+        LOG.error("Error storing AMRMProxy application context entry for "
+            + getApplicationContext().getApplicationAttemptId(), e);
+      }
+    }
+
     // the queue this application belongs will be used for getting
     // AMRMProxy policy from state store.
     String queue = this.amRegistrationResponse.getQueue();
@@ -437,6 +621,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           if (uamResponse.getResponse() == null
               || !uamResponse.getResponse().getIsUnregistered()) {
             failedToUnRegister = true;
+          } else if (getNMStateStore() != null) {
+            getNMStateStore().removeAMRMProxyAppContextEntry(
+                getApplicationContext().getApplicationAttemptId(),
+                NMSS_SECONDARY_SC_PREFIX + uamResponse.getSubClusterId());
           }
         } catch (Throwable e) {
           failedToUnRegister = true;
@@ -496,6 +684,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     }
   }
 
+  @VisibleForTesting
+  protected FederationRegistryClient getRegistryClient() {
+    return this.registryClient;
+  }
+
   /**
    * Create the UAM pool manager for secondary sub-clsuters. For unit test to
    * override.
@@ -510,18 +703,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   }
 
   /**
-   * Returns instance of the ApplicationMasterProtocol proxy class that is used
-   * to connect to the Home resource manager.
+   * Create a proxy instance that is used to connect to the Home resource
+   * manager.
    *
    * @param appContext AMRMProxyApplicationContext
+   * @param protocol the protocol class for the proxy
+   * @param user the ugi for the proxy
+   * @param <T> the type of the proxy
    * @return the proxy created
    */
-  protected ApplicationMasterProtocol createHomeRMProxy(
-      AMRMProxyApplicationContext appContext) {
+  protected <T> T createHomeRMProxy(AMRMProxyApplicationContext appContext,
+      Class<T> protocol, UserGroupInformation user) {
     try {
       return FederationProxyProviderUtil.createRMProxy(appContext.getConf(),
-          ApplicationMasterProtocol.class, this.homeSubClusterId, this.appOwner,
-          appContext.getAMRMToken());
+          protocol, this.homeSubClusterId, user, appContext.getAMRMToken());
     } catch (Exception ex) {
       throw new YarnRuntimeException(ex);
     }
@@ -810,17 +1005,31 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 responses.add(response);
               }
 
-              // Save the new AMRMToken for the UAM in registry if present
+              // Save the new AMRMToken for the UAM if present
               if (response.getAMRMToken() != null) {
                 Token<AMRMTokenIdentifier> newToken = ConverterUtils
                     .convertFromYarn(response.getAMRMToken(), (Text) null);
-                // Update the token in registry
+                // Update the token in registry or NMSS
                 if (registryClient != null) {
                   registryClient
                       .writeAMRMTokenForUAM(
                           getApplicationContext().getApplicationAttemptId()
                               .getApplicationId(),
                           subClusterId.getId(), newToken);
+                } else if (getNMStateStore() != null) {
+                  try {
+                    getNMStateStore().storeAMRMProxyAppContextEntry(
+                        getApplicationContext().getApplicationAttemptId(),
+                        NMSS_SECONDARY_SC_PREFIX + subClusterId.getId(),
+                        newToken.encodeToUrlString()
+                            .getBytes(STRING_TO_BYTE_FORMAT));
+                  } catch (IOException e) {
+                    LOG.error(
+                        "Error storing UAM token as AMRMProxy "
+                            + "context entry in NMSS for "
+                            + getApplicationContext().getApplicationAttemptId(),
+                        e);
+                  }
                 }
               }
 
@@ -925,12 +1134,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
             successfulRegistrations.put(uamResponse.getSubClusterId(),
                 uamResponse.getResponse());
 
+            // Save the UAM token in registry or NMSS
             if (registryClient != null) {
               registryClient.writeAMRMTokenForUAM(
                   getApplicationContext().getApplicationAttemptId()
                       .getApplicationId(),
                   uamResponse.getSubClusterId().getId(),
                   uamResponse.getUamToken());
+            } else if (getNMStateStore() != null) {
+              getNMStateStore().storeAMRMProxyAppContextEntry(
+                  getApplicationContext().getApplicationAttemptId(),
+                  NMSS_SECONDARY_SC_PREFIX
+                      + uamResponse.getSubClusterId().getId(),
+                  uamResponse.getUamToken().encodeToUrlString()
+                      .getBytes(STRING_TO_BYTE_FORMAT));
             }
           }
         } catch (Exception e) {
@@ -952,11 +1169,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private AllocateResponse mergeAllocateResponses(
       AllocateResponse homeResponse) {
     // Timing issue, we need to remove the completed and then save the new ones.
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Remove containers: "
-          + homeResponse.getCompletedContainersStatuses());
-      LOG.debug("Adding containers: " + homeResponse.getAllocatedContainers());
-    }
     removeFinishedContainersFromCache(
         homeResponse.getCompletedContainersStatuses());
     cacheAllocatedContainers(homeResponse.getAllocatedContainers(),
@@ -989,6 +1201,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private void removeFinishedContainersFromCache(
       List<ContainerStatus> finishedContainers) {
     for (ContainerStatus container : finishedContainers) {
+      LOG.debug("Completed container {}", container);
       if (containerIdToSubClusterIdMap
           .containsKey(container.getContainerId())) {
         containerIdToSubClusterIdMap.remove(container.getContainerId());
@@ -1146,12 +1359,22 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private void cacheAllocatedContainers(List<Container> containers,
       SubClusterId subClusterId) {
     for (Container container : containers) {
+      LOG.debug("Adding container {}", container);
       if (containerIdToSubClusterIdMap.containsKey(container.getId())) {
         SubClusterId existingSubClusterId =
             containerIdToSubClusterIdMap.get(container.getId());
         if (existingSubClusterId.equals(subClusterId)) {
-          // When RM fails over, the new RM master might send out the same
-          // container allocation more than once. Just move on in this case.
+          /*
+           * When RM fails over, the new RM master might send out the same
+           * container allocation more than once.
+           *
+           * It is also possible because of a recent NM restart with NM recovery
+           * enabled. We recover running containers from RM. But RM might not
+           * notified AM of some of these containers yet. When RM dose notify,
+           * we will already have these containers in the map.
+           *
+           * Either case, just warn and move on.
+           */
           LOG.warn(
               "Duplicate containerID: {} found in the allocated containers"
                   + " from same sub-cluster: {}, so ignoring.",
@@ -1226,7 +1449,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   private boolean warnIfNotExists(ContainerId containerId, String actionName) {
     if (!this.containerIdToSubClusterIdMap.containsKey(containerId)) {
-      LOG.error("AM is trying to {} a container {} that does not exist. ",
+      LOG.error(
+          "AM is trying to {} a container {} that does not exist. Might happen "
+              + "shortly after NM restart when NM recovery is enabled",
           actionName, containerId.toString());
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index da1d047..0319dbe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredAMRMProxyState;
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
@@ -180,6 +181,20 @@ public abstract class BaseAMRMProxyTest {
     return new NMContext(null, null, null, null, stateStore, false, this.conf);
   }
 
+  // A utility method for intercepter recover unit test
+  protected Map<String, byte[]> recoverDataMapForAppAttempt(
+      NMStateStoreService nmStateStore, ApplicationAttemptId attemptId)
+      throws IOException {
+    RecoveredAMRMProxyState state = nmStateStore.loadAMRMProxyState();
+    for (Map.Entry<ApplicationAttemptId, Map<String, byte[]>> entry : state
+        .getAppContexts().entrySet()) {
+      if (entry.getKey().equals(attemptId)) {
+        return entry.getValue();
+      }
+    }
+    return null;
+  }
+
   protected List<ContainerId> getCompletedContainerIds(
       List<ContainerStatus> containerStatus) {
     List<ContainerId> ret = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
index aa7ed69..eefaba1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
@@ -450,6 +450,104 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
   }
 
   @Test
+  public void testRecoverWithAMRMProxyHA() throws Exception {
+    testRecover(registry);
+  }
+
+  @Test
+  public void testRecoverWithoutAMRMProxyHA() throws Exception {
+    testRecover(null);
+  }
+
+  public void testRecover(RegistryOperations registryObj) throws Exception {
+    ApplicationUserInfo userInfo = getApplicationUserInfo(testAppId);
+    userInfo.getUser().doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        interceptor = new TestableFederationInterceptor();
+        interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
+            getConf(), attemptId, "test-user", null, null, null, registryObj));
+        interceptor.cleanupRegistry();
+
+        // Register the application
+        RegisterApplicationMasterRequest registerReq =
+            Records.newRecord(RegisterApplicationMasterRequest.class);
+        registerReq.setHost(Integer.toString(testAppId));
+        registerReq.setRpcPort(testAppId);
+        registerReq.setTrackingUrl("");
+
+        RegisterApplicationMasterResponse registerResponse =
+            interceptor.registerApplicationMaster(registerReq);
+        Assert.assertNotNull(registerResponse);
+
+        Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
+
+        // Allocate one batch of containers
+        registerSubCluster(SubClusterId.newInstance("SC-1"));
+        registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
+
+        int numberOfContainers = 3;
+        List<Container> containers =
+            getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
+        Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
+
+        // Prepare for Federation Interceptor restart and recover
+        Map<String, byte[]> recoveredDataMap =
+            recoverDataMapForAppAttempt(nmStateStore, attemptId);
+        String scEntry =
+            FederationInterceptor.NMSS_SECONDARY_SC_PREFIX + "SC-1";
+        if (registryObj == null) {
+          Assert.assertTrue(recoveredDataMap.containsKey(scEntry));
+        } else {
+          // When AMRMPRoxy HA is enabled, NMSS should not have the UAM token,
+          // it should be in Registry
+          Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
+        }
+
+        // Preserve the mock RM instances
+        MockResourceManagerFacade homeRM = interceptor.getHomeRM();
+        ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
+            interceptor.getSecondaryRMs();
+
+        // Create a new intercepter instance and recover
+        interceptor = new TestableFederationInterceptor(homeRM, secondaries);
+        interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
+            getConf(), attemptId, "test-user", null, null, null, registryObj));
+        interceptor.recover(recoveredDataMap);
+
+        Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
+
+        // Release all containers
+        releaseContainersAndAssert(containers);
+
+        // Finish the application
+        FinishApplicationMasterRequest finishReq =
+            Records.newRecord(FinishApplicationMasterRequest.class);
+        finishReq.setDiagnostics("");
+        finishReq.setTrackingUrl("");
+        finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
+
+        FinishApplicationMasterResponse finshResponse =
+            interceptor.finishApplicationMaster(finishReq);
+        Assert.assertNotNull(finshResponse);
+        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+
+        // After the application succeeds, the registry/NMSS entry should be
+        // cleaned up
+        if (registryObj != null) {
+          Assert.assertEquals(0,
+              interceptor.getRegistryClient().getAllApplications().size());
+        } else {
+          recoveredDataMap =
+              recoverDataMapForAppAttempt(nmStateStore, attemptId);
+          Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
+        }
+        return null;
+      }
+    });
+  }
+
+  @Test
   public void testRequestInterceptorChainCreation() throws Exception {
     RequestInterceptor root =
         super.getAMRMProxyService().createRequestInterceptorChain();
@@ -636,6 +734,12 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
             interceptor.finishApplicationMaster(finishReq);
         Assert.assertNotNull(finshResponse);
         Assert.assertEquals(true, finshResponse.getIsUnregistered());
+
+        // After the application succeeds, the registry entry should be deleted
+        if (interceptor.getRegistryClient() != null) {
+          Assert.assertEquals(0,
+              interceptor.getRegistryClient().getAllApplications().size());
+        }
         return null;
       }
     });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/670e8d4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
index 23c80ae..1088c69 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
@@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
@@ -59,16 +58,17 @@ public class TestableFederationInterceptor extends FederationInterceptor {
     return new TestableUnmanagedAMPoolManager(threadPool);
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  protected ApplicationMasterProtocol createHomeRMProxy(
-      AMRMProxyApplicationContext appContext) {
+  protected <T> T createHomeRMProxy(AMRMProxyApplicationContext appContext,
+      Class<T> protocol, UserGroupInformation user) {
     synchronized (this) {
       if (mockRm == null) {
         mockRm = new MockResourceManagerFacade(
             new YarnConfiguration(super.getConf()), 0);
       }
     }
-    return mockRm;
+    return (T) mockRm;
   }
 
   @SuppressWarnings("unchecked")


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


[17/50] [abbrv] hadoop git commit: HDFS-11289. [SPS]: Make SPS movement monitor timeouts configurable. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11289. [SPS]: Make SPS movement monitor timeouts configurable. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: aa6bf50bc82b4a43a86b4a1c44dd169b237abdd8
Parents: 470e7f7
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Mon Jan 9 19:07:43 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:40 2017 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  9 ++++++++
 .../server/blockmanagement/BlockManager.java    |  4 ++--
 .../BlockStorageMovementAttemptedItems.java     | 10 ++++-----
 .../server/namenode/StoragePolicySatisfier.java | 15 ++++++++-----
 .../src/main/resources/hdfs-default.xml         | 23 ++++++++++++++++++++
 5 files changed, 49 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa6bf50b/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 288cc2b..6b5b69b 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
@@ -569,10 +569,19 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_MOVER_MAX_NO_MOVE_INTERVAL_KEY = "dfs.mover.max-no-move-interval";
   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_RECHECK_TIMEOUT_MILLIS_KEY =
+      "dfs.storage.policy.satisfier.recheck.timeout.millis";
+  public static final int DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT =
+      5 * 60 * 1000;
+  public static final String DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY =
+      "dfs.storage.policy.satisfier.self.retry.timeout.millis";
+  public static final int DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT =
+      30 * 60 * 1000;
 
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 9866;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa6bf50b/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 74ee4b5..472a537 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
@@ -480,8 +480,8 @@ public class BlockManager implements BlockStatsMXBean {
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_DEFAULT);
     if (storagePolicyEnabled && spsEnabled) {
-      sps = new StoragePolicySatisfier(namesystem,
-          storageMovementNeeded, this);
+      sps = new StoragePolicySatisfier(namesystem, storageMovementNeeded, this,
+          conf);
     } else {
       sps = null;
       LOG.warn(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa6bf50b/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 ce97075..042aca3 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
@@ -64,14 +64,14 @@ public class BlockStorageMovementAttemptedItems {
   // It might take anywhere between 5 to 10 minutes before
   // a request is timed out.
   //
-  private long checkTimeout = 5 * 60 * 1000; // minimum value
+  private long minCheckTimeout = 5 * 60 * 1000; // minimum value
   private BlockStorageMovementNeeded blockStorageMovementNeeded;
 
-  public BlockStorageMovementAttemptedItems(long timeoutPeriod,
+  public BlockStorageMovementAttemptedItems(long recheckTimeout,
       long selfRetryTimeout,
       BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
-    if (timeoutPeriod > 0) {
-      this.checkTimeout = Math.min(checkTimeout, timeoutPeriod);
+    if (recheckTimeout > 0) {
+      this.minCheckTimeout = Math.min(minCheckTimeout, recheckTimeout);
     }
 
     this.selfRetryTimeout = selfRetryTimeout;
@@ -196,7 +196,7 @@ public class BlockStorageMovementAttemptedItems {
         try {
           blockStorageMovementResultCheck();
           blocksStorageMovementUnReportedItemsCheck();
-          Thread.sleep(checkTimeout);
+          Thread.sleep(minCheckTimeout);
         } catch (InterruptedException ie) {
           LOG.info("BlocksStorageMovementAttemptResultMonitor thread "
               + "is interrupted.", ie);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa6bf50b/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 a854bd7..ee59617 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
@@ -27,7 +27,9 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -79,15 +81,18 @@ public class StoragePolicySatisfier implements Runnable {
 
   public StoragePolicySatisfier(final Namesystem namesystem,
       final BlockStorageMovementNeeded storageMovementNeeded,
-      final BlockManager blkManager) {
+      final BlockManager blkManager, Configuration conf) {
     this.namesystem = namesystem;
     this.storageMovementNeeded = storageMovementNeeded;
     this.blockManager = blkManager;
-    // TODO: below selfRetryTimeout and checkTimeout can be configurable later
-    // Now, the default values of selfRetryTimeout and checkTimeout are 30mins
-    // and 5mins respectively
     this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(
-        5 * 60 * 1000, 30 * 60 * 1000, storageMovementNeeded);
+        conf.getLong(
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT),
+        conf.getLong(
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
+        storageMovementNeeded);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa6bf50b/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 71c5e3a..8beadd2 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
@@ -4358,6 +4358,29 @@
 </property>
 
 <property>
+  <name>dfs.storage.policy.satisfier.recheck.timeout.millis</name>
+  <value>300000</value>
+  <description>
+    Blocks storage movements monitor re-check interval in milliseconds.
+    This check will verify whether any blocks storage movement results arrived from DN
+    and also verify if any of file blocks movements not at all reported to DN
+    since dfs.storage.policy.satisfier.self.retry.timeout.
+    The default value is 5 * 60 * 1000 (5 mins)
+  </description>
+</property>
+
+<property>
+  <name>dfs.storage.policy.satisfier.self.retry.timeout.millis</name>
+  <value>1800000</value>
+  <description>
+    If any of file related block movements not at all reported by coordinator datanode,
+    then after this timeout(in milliseconds), the item will be added back to movement needed list
+    at namenode which will be retried for block movements.
+    The default value is 30 * 60 * 1000 (30 mins)
+  </description>
+</property>
+
+<property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>
   <description>


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


[42/50] [abbrv] hadoop git commit: HDFS-12291: [SPS]: Provide a mechanism to recursively iterate and satisfy storage policy of all the files under the given dir. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 3375590..57e9f94 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
@@ -21,6 +21,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KE
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.slf4j.LoggerFactory.getLogger;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -61,8 +64,10 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
 
 import com.google.common.base.Supplier;
 
@@ -71,6 +76,12 @@ import com.google.common.base.Supplier;
  * moved and finding its suggested target locations to move.
  */
 public class TestStoragePolicySatisfier {
+
+  {
+    GenericTestUtils.setLogLevel(
+        getLogger(FSTreeTraverser.class), Level.DEBUG);
+  }
+
   private static final String ONE_SSD = "ONE_SSD";
   private static final String COLD = "COLD";
   private static final Logger LOG =
@@ -341,7 +352,9 @@ public class TestStoragePolicySatisfier {
 
       // take no effect for the sub-dir's file in the directory.
       DFSTestUtil.waitExpectedStorageType(
-          subFile2, StorageType.DEFAULT, 3, 30000, dfs);
+          subFile2, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          subFile2, StorageType.DISK, 2, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -1083,6 +1096,368 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Test SPS for empty directory, xAttr should be removed.
+   */
+  @Test(timeout = 300000)
+  public void testSPSForEmptyDirectory() throws IOException, TimeoutException,
+      InterruptedException {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path emptyDir = new Path("/emptyDir");
+      fs.mkdirs(emptyDir);
+      fs.satisfyStoragePolicy(emptyDir);
+      // Make sure satisfy xattr has been removed.
+      DFSTestUtil.waitForXattrRemoved("/emptyDir",
+          XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test SPS for not exist directory.
+   */
+  @Test(timeout = 300000)
+  public void testSPSForNonExistDirectory() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path emptyDir = new Path("/emptyDir");
+      try {
+        fs.satisfyStoragePolicy(emptyDir);
+        fail("FileNotFoundException should throw");
+      } catch (FileNotFoundException e) {
+        // nothing to do
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test SPS for directory tree which doesn't have files.
+   */
+  @Test(timeout = 300000)
+  public void testSPSWithDirectoryTreeWithoutFile() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      // Create directories
+      /*
+       *                   root
+       *                    |
+       *           A--------C--------D
+       *                    |
+       *               G----H----I
+       *                    |
+       *                    O
+       */
+      DistributedFileSystem fs = cluster.getFileSystem();
+      fs.mkdirs(new Path("/root/C/H/O"));
+      fs.mkdirs(new Path("/root/A"));
+      fs.mkdirs(new Path("/root/D"));
+      fs.mkdirs(new Path("/root/C/G"));
+      fs.mkdirs(new Path("/root/C/I"));
+      fs.satisfyStoragePolicy(new Path("/root"));
+      // Make sure satisfy xattr has been removed.
+      DFSTestUtil.waitForXattrRemoved("/root",
+          XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test SPS for directory which has multilevel directories.
+   */
+  @Test(timeout = 300000)
+  public void testMultipleLevelDirectoryForSatisfyStoragePolicy()
+      throws Exception {
+    try {
+      StorageType[][] diskTypes = new StorageType[][] {
+          {StorageType.DISK, StorageType.ARCHIVE},
+          {StorageType.ARCHIVE, StorageType.SSD},
+          {StorageType.DISK, StorageType.DISK}};
+      config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+          storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      createDirectoryTree(dfs);
+
+      List<String> files = getDFSListOfTree();
+      dfs.setStoragePolicy(new Path("/root"), COLD);
+      dfs.satisfyStoragePolicy(new Path("/root"));
+      for (String fileName : files) {
+        // Wait till the block is moved to ARCHIVE
+        DFSTestUtil.waitExpectedStorageType(fileName, StorageType.ARCHIVE, 2,
+            30000, dfs);
+      }
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Test SPS for batch processing.
+   */
+  @Test(timeout = 300000)
+  public void testBatchProcessingForSPSDirectory() throws Exception {
+    try {
+      StorageType[][] diskTypes = new StorageType[][] {
+          {StorageType.DISK, StorageType.ARCHIVE},
+          {StorageType.ARCHIVE, StorageType.SSD},
+          {StorageType.DISK, StorageType.DISK}};
+      config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      // Set queue max capacity
+      config.setInt(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
+          5);
+      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+          storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      createDirectoryTree(dfs);
+      List<String> files = getDFSListOfTree();
+      LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(LogFactory
+          .getLog(FSTreeTraverser.class));
+
+      dfs.setStoragePolicy(new Path("/root"), COLD);
+      dfs.satisfyStoragePolicy(new Path("/root"));
+      for (String fileName : files) {
+        // Wait till the block is moved to ARCHIVE
+        DFSTestUtil.waitExpectedStorageType(fileName, StorageType.ARCHIVE, 2,
+            30000, dfs);
+      }
+      waitForBlocksMovementResult(files.size(), 30000);
+      String expectedLogMessage = "StorageMovementNeeded queue remaining"
+          + " capacity is zero";
+      assertTrue("Log output does not contain expected log message: "
+          + expectedLogMessage, logs.getOutput().contains(expectedLogMessage));
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+
+  /**
+   *  Test traverse when parent got deleted.
+   *  1. Delete /root when traversing Q
+   *  2. U, R, S should not be in queued.
+   */
+  @Test
+  public void testTraverseWhenParentDeleted() throws Exception {
+    StorageType[][] diskTypes = new StorageType[][] {
+        {StorageType.DISK, StorageType.ARCHIVE},
+        {StorageType.ARCHIVE, StorageType.SSD},
+        {StorageType.DISK, StorageType.DISK}};
+    config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+        storagesPerDatanode, capacity);
+    dfs = hdfsCluster.getFileSystem();
+    createDirectoryTree(dfs);
+
+    List<String> expectedTraverseOrder = getDFSListOfTree();
+
+    //Remove files which will not be traverse when parent is deleted
+    expectedTraverseOrder.remove("/root/D/L/R");
+    expectedTraverseOrder.remove("/root/D/L/S");
+    expectedTraverseOrder.remove("/root/D/L/Q/U");
+    FSDirectory fsDir = hdfsCluster.getNamesystem().getFSDirectory();
+
+    //Queue limit can control the traverse logic to wait for some free
+    //entry in queue. After 10 files, traverse control will be on U.
+    StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
+    Mockito.when(sps.isRunning()).thenReturn(true);
+    BlockStorageMovementNeeded movmentNeededQueue =
+        new BlockStorageMovementNeeded(fsDir.getFSNamesystem(), sps, 10);
+    INode rootINode = fsDir.getINode("/root");
+    movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
+    movmentNeededQueue.init();
+
+    //Wait for thread to reach U.
+    Thread.sleep(1000);
+
+    dfs.delete(new Path("/root/D/L"), true);
+
+    // Remove 10 element and make queue free, So other traversing will start.
+    for (int i = 0; i < 10; i++) {
+      String path = expectedTraverseOrder.remove(0);
+      long trackId = movmentNeededQueue.get().getTrackId();
+      INode inode = fsDir.getInode(trackId);
+      assertTrue("Failed to traverse tree, expected " + path + " but got "
+          + inode.getFullPathName(), path.equals(inode.getFullPathName()));
+    }
+    //Wait to finish tree traverse
+    Thread.sleep(5000);
+
+    // Check other element traversed in order and R,S should not be added in
+    // queue which we already removed from expected list
+    for (String path : expectedTraverseOrder) {
+      long trackId = movmentNeededQueue.get().getTrackId();
+      INode inode = fsDir.getInode(trackId);
+      assertTrue("Failed to traverse tree, expected " + path + " but got "
+          + inode.getFullPathName(), path.equals(inode.getFullPathName()));
+    }
+    dfs.delete(new Path("/root"), true);
+  }
+
+  /**
+   *  Test traverse when root parent got deleted.
+   *  1. Delete L when traversing Q
+   *  2. E, M, U, R, S should not be in queued.
+   */
+  @Test
+  public void testTraverseWhenRootParentDeleted() throws Exception {
+    StorageType[][] diskTypes = new StorageType[][] {
+        {StorageType.DISK, StorageType.ARCHIVE},
+        {StorageType.ARCHIVE, StorageType.SSD},
+        {StorageType.DISK, StorageType.DISK}};
+    config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+        storagesPerDatanode, capacity);
+    dfs = hdfsCluster.getFileSystem();
+    createDirectoryTree(dfs);
+
+    List<String> expectedTraverseOrder = getDFSListOfTree();
+
+    // Remove files which will not be traverse when parent is deleted
+    expectedTraverseOrder.remove("/root/D/L/R");
+    expectedTraverseOrder.remove("/root/D/L/S");
+    expectedTraverseOrder.remove("/root/D/L/Q/U");
+    expectedTraverseOrder.remove("/root/D/M");
+    expectedTraverseOrder.remove("/root/E");
+    FSDirectory fsDir = hdfsCluster.getNamesystem().getFSDirectory();
+    StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
+    Mockito.when(sps.isRunning()).thenReturn(true);
+    // Queue limit can control the traverse logic to wait for some free
+    // entry in queue. After 10 files, traverse control will be on U.
+    BlockStorageMovementNeeded movmentNeededQueue =
+        new BlockStorageMovementNeeded(fsDir.getFSNamesystem(), sps, 10);
+    movmentNeededQueue.init();
+    INode rootINode = fsDir.getINode("/root");
+    movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
+    // Wait for thread to reach U.
+    Thread.sleep(1000);
+
+    dfs.delete(new Path("/root/D/L"), true);
+
+    // Remove 10 element and make queue free, So other traversing will start.
+    for (int i = 0; i < 10; i++) {
+      String path = expectedTraverseOrder.remove(0);
+      long trackId = movmentNeededQueue.get().getTrackId();
+      INode inode = fsDir.getInode(trackId);
+      assertTrue("Failed to traverse tree, expected " + path + " but got "
+          + inode.getFullPathName(), path.equals(inode.getFullPathName()));
+    }
+    // Wait to finish tree traverse
+    Thread.sleep(5000);
+
+    // Check other element traversed in order and E, M, U, R, S should not be
+    // added in queue which we already removed from expected list
+    for (String path : expectedTraverseOrder) {
+      long trackId = movmentNeededQueue.get().getTrackId();
+      INode inode = fsDir.getInode(trackId);
+      assertTrue("Failed to traverse tree, expected " + path + " but got "
+          + inode.getFullPathName(), path.equals(inode.getFullPathName()));
+    }
+    dfs.delete(new Path("/root"), true);
+  }
+
+  private static void createDirectoryTree(DistributedFileSystem dfs)
+      throws Exception {
+    // tree structure
+    /*
+     *                           root
+     *                             |
+     *           A--------B--------C--------D--------E
+     *                    |                 |
+     *          F----G----H----I       J----K----L----M
+     *               |                           |
+     *          N----O----P                 Q----R----S
+     *                    |                 |
+     *                    T                 U
+     */
+    // create root Node and child
+    dfs.mkdirs(new Path("/root"));
+    DFSTestUtil.createFile(dfs, new Path("/root/A"), 1024, (short) 3, 0);
+    dfs.mkdirs(new Path("/root/B"));
+    DFSTestUtil.createFile(dfs, new Path("/root/C"), 1024, (short) 3, 0);
+    dfs.mkdirs(new Path("/root/D"));
+    DFSTestUtil.createFile(dfs, new Path("/root/E"), 1024, (short) 3, 0);
+
+    // Create /root/B child
+    DFSTestUtil.createFile(dfs, new Path("/root/B/F"), 1024, (short) 3, 0);
+    dfs.mkdirs(new Path("/root/B/G"));
+    DFSTestUtil.createFile(dfs, new Path("/root/B/H"), 1024, (short) 3, 0);
+    DFSTestUtil.createFile(dfs, new Path("/root/B/I"), 1024, (short) 3, 0);
+
+    // Create /root/D child
+    DFSTestUtil.createFile(dfs, new Path("/root/D/J"), 1024, (short) 3, 0);
+    DFSTestUtil.createFile(dfs, new Path("/root/D/K"), 1024, (short) 3, 0);
+    dfs.mkdirs(new Path("/root/D/L"));
+    DFSTestUtil.createFile(dfs, new Path("/root/D/M"), 1024, (short) 3, 0);
+
+    // Create /root/B/G child
+    DFSTestUtil.createFile(dfs, new Path("/root/B/G/N"), 1024, (short) 3, 0);
+    DFSTestUtil.createFile(dfs, new Path("/root/B/G/O"), 1024, (short) 3, 0);
+    dfs.mkdirs(new Path("/root/B/G/P"));
+
+    // Create /root/D/L child
+    dfs.mkdirs(new Path("/root/D/L/Q"));
+    DFSTestUtil.createFile(dfs, new Path("/root/D/L/R"), 1024, (short) 3, 0);
+    DFSTestUtil.createFile(dfs, new Path("/root/D/L/S"), 1024, (short) 3, 0);
+
+    // Create /root/B/G/P child
+    DFSTestUtil.createFile(dfs, new Path("/root/B/G/P/T"), 1024, (short) 3, 0);
+
+    // Create /root/D/L/Q child
+    DFSTestUtil.createFile(dfs, new Path("/root/D/L/Q/U"), 1024, (short) 3, 0);
+  }
+
+  private List<String> getDFSListOfTree() {
+    List<String> dfsList = new ArrayList<>();
+    dfsList.add("/root/A");
+    dfsList.add("/root/B/F");
+    dfsList.add("/root/B/G/N");
+    dfsList.add("/root/B/G/O");
+    dfsList.add("/root/B/G/P/T");
+    dfsList.add("/root/B/H");
+    dfsList.add("/root/B/I");
+    dfsList.add("/root/C");
+    dfsList.add("/root/D/J");
+    dfsList.add("/root/D/K");
+    dfsList.add("/root/D/L/Q/U");
+    dfsList.add("/root/D/L/R");
+    dfsList.add("/root/D/L/S");
+    dfsList.add("/root/D/M");
+    dfsList.add("/root/E");
+    return dfsList;
+  }
+
   private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
       throws IOException {
     ArrayList<DataNode> dns = hdfsCluster.getDataNodes();


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


[24/50] [abbrv] hadoop git commit: HDFS-11336: [SPS]: Remove xAttrs when movements done or SPS disabled. Contributed by Yuanbo Liu.

Posted by ra...@apache.org.
HDFS-11336: [SPS]: Remove xAttrs when movements done or SPS disabled. Contributed by Yuanbo Liu.


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

Branch: refs/heads/HDFS-10285
Commit: cbbe71f8cea83ce8bf1f8df4e3593ca46879c7c3
Parents: f8efef5
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue Mar 14 00:52:24 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:17 2017 +0530

----------------------------------------------------------------------
 .../BlockStorageMovementAttemptedItems.java     |  14 ++-
 .../hdfs/server/namenode/FSDirAttrOp.java       |   8 ++
 .../hdfs/server/namenode/FSDirectory.java       |  16 +++
 .../server/namenode/StoragePolicySatisfier.java |  45 ++++++--
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   2 +-
 .../TestBlockStorageMovementAttemptedItems.java |   6 +-
 .../TestPersistentStoragePolicySatisfier.java   | 112 ++++++++++++++++++-
 7 files changed, 186 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbbe71f8/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 042aca3..f15db73 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.util.Time.monotonicNow;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -54,6 +55,7 @@ public class BlockStorageMovementAttemptedItems {
   private final List<BlocksStorageMovementResult> storageMovementAttemptedResults;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
+  private final StoragePolicySatisfier sps;
   //
   // It might take anywhere between 30 to 60 minutes before
   // a request is timed out.
@@ -69,7 +71,8 @@ public class BlockStorageMovementAttemptedItems {
 
   public BlockStorageMovementAttemptedItems(long recheckTimeout,
       long selfRetryTimeout,
-      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
+      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles,
+      StoragePolicySatisfier sps) {
     if (recheckTimeout > 0) {
       this.minCheckTimeout = Math.min(minCheckTimeout, recheckTimeout);
     }
@@ -78,6 +81,7 @@ public class BlockStorageMovementAttemptedItems {
     this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
     storageMovementAttemptedItems = new HashMap<>();
     storageMovementAttemptedResults = new ArrayList<>();
+    this.sps = sps;
   }
 
   /**
@@ -200,6 +204,9 @@ public class BlockStorageMovementAttemptedItems {
         } catch (InterruptedException ie) {
           LOG.info("BlocksStorageMovementAttemptResultMonitor thread "
               + "is interrupted.", ie);
+        } catch (IOException ie) {
+          LOG.warn("BlocksStorageMovementAttemptResultMonitor thread "
+              + "received exception and exiting.", ie);
         }
       }
     }
@@ -248,7 +255,7 @@ public class BlockStorageMovementAttemptedItems {
   }
 
   @VisibleForTesting
-  void blockStorageMovementResultCheck() {
+  void blockStorageMovementResultCheck() throws IOException {
     synchronized (storageMovementAttemptedResults) {
       Iterator<BlocksStorageMovementResult> resultsIter =
           storageMovementAttemptedResults.iterator();
@@ -296,6 +303,9 @@ public class BlockStorageMovementAttemptedItems {
                   + " reported from co-ordinating datanode. But the trackID "
                   + "doesn't exists in storageMovementAttemptedItems list",
                   storageMovementAttemptedResult.getTrackId());
+              // Remove xattr for the track id.
+              this.sps.notifyBlkStorageMovementFinished(
+                  storageMovementAttemptedResult.getTrackId());
             }
           }
           // Remove trackID from the attempted list, if any.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbbe71f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 991e855..0132f36 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -541,6 +541,14 @@ public class FSDirAttrOp {
     return false;
   }
 
+  static void unprotectedRemoveSPSXAttr(INode inode, XAttr spsXAttr)
+      throws IOException{
+    List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
+    existingXAttrs.remove(spsXAttr);
+    XAttrStorage.updateINodeXAttrs(inode, existingXAttrs,
+        INodesInPath.fromINode(inode).getLatestSnapshotId());
+  }
+
   private static void setDirStoragePolicy(
       FSDirectory fsd, INodesInPath iip, byte policyId) throws IOException {
     INode inode = FSDirectory.resolveLastINode(iip);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbbe71f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 37ed027..89b8ec3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -1416,6 +1416,22 @@ public class FSDirectory implements Closeable {
     getBlockManager().satisfyStoragePolicy(inode.getId());
   }
 
+  /**
+   * Remove the SPS xattr from the inode, retrieve the inode from the
+   * block collection id.
+   * @param id
+   *           - file block collection id.
+   */
+  public void removeSPSXattr(long id) throws IOException {
+    final INode inode = getInode(id);
+    final XAttrFeature xaf = inode.getXAttrFeature();
+    final XAttr spsXAttr = xaf.getXAttr(XATTR_SATISFY_STORAGE_POLICY);
+
+    if (spsXAttr != null) {
+      FSDirAttrOp.unprotectedRemoveSPSXAttr(inode, spsXAttr);
+    }
+  }
+
   private void addEncryptionZone(INodeWithAdditionalFields inode,
       XAttrFeature xaf) {
     if (xaf == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbbe71f8/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 29c8a5d..337d5b5 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -91,7 +92,8 @@ public class StoragePolicySatisfier implements Runnable {
         conf.getLong(
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
-        storageMovementNeeded);
+        storageMovementNeeded,
+        this);
   }
 
   /**
@@ -119,12 +121,6 @@ public class StoragePolicySatisfier implements Runnable {
    */
   public synchronized void stop(boolean reconfigStop) {
     isRunning = false;
-    if (reconfigStop) {
-      LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
-          + "deactivate it.");
-    } else {
-      LOG.info("Stopping StoragePolicySatisfier.");
-    }
     if (storagePolicySatisfierThread == null) {
       return;
     }
@@ -135,8 +131,12 @@ public class StoragePolicySatisfier implements Runnable {
     }
     this.storageMovementsMonitor.stop();
     if (reconfigStop) {
-      this.clearQueues();
+      LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
+          + "deactivate it.");
+      this.clearQueuesWithNotification();
       this.blockManager.getDatanodeManager().addDropSPSWorkCommandsToAllDNs();
+    } else {
+      LOG.info("Stopping StoragePolicySatisfier.");
     }
   }
 
@@ -717,4 +717,33 @@ public class StoragePolicySatisfier implements Runnable {
         + "user requests on satisfying block storages would be discarded.");
     storageMovementNeeded.clearAll();
   }
+
+  /**
+   * Clean all the movements in storageMovementNeeded and notify
+   * to clean up required resources.
+   * @throws IOException
+   */
+  private void clearQueuesWithNotification() {
+    Long id;
+    while ((id = storageMovementNeeded.get()) != null) {
+      try {
+        notifyBlkStorageMovementFinished(id);
+      } catch (IOException ie) {
+        LOG.warn("Failed to remove SPS "
+            + "xattr for collection id " + id, ie);
+      }
+    }
+  }
+
+  /**
+   * When block movement has been finished successfully, some additional
+   * operations should be notified, for example, SPS xattr should be
+   * removed.
+   * @param trackId track id i.e., block collection id.
+   * @throws IOException
+   */
+  public void notifyBlkStorageMovementFinished(long trackId)
+      throws IOException {
+    this.namesystem.getFSDirectory().removeSPSXattr(trackId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbbe71f8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index d04b8e4..1cec9b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -2389,6 +2389,6 @@ public class DFSTestUtil {
                 + expectedStorageCount + " and actual=" + actualStorageCount);
         return expectedStorageCount == actualStorageCount;
       }
-    }, 1000, timeout);
+    }, 500, timeout);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbbe71f8/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 6641134..95142d3 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
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 /**
  * Tests that block storage movement attempt failures are reported from DN and
@@ -36,10 +37,11 @@ public class TestBlockStorageMovementAttemptedItems {
   private final int selfRetryTimeout = 500;
 
   @Before
-  public void setup() {
+  public void setup() throws Exception {
     unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded();
+    StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
     bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
-        selfRetryTimeout, unsatisfiedStorageMovementFiles);
+        selfRetryTimeout, unsatisfiedStorageMovementFiles, sps);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cbbe71f8/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 e4b4290..8c3359a 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
@@ -20,16 +20,22 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+import static org.junit.Assert.assertFalse;
 
 /**
  * Test persistence of satisfying files/directories.
@@ -72,7 +78,16 @@ public class TestPersistentStoragePolicySatisfier {
    * @throws IOException
    */
   public void clusterSetUp() throws Exception {
-    clusterSetUp(false);
+    clusterSetUp(false, new HdfsConfiguration());
+  }
+
+  /**
+   * Setup environment for every test case.
+   * @param hdfsConf hdfs conf.
+   * @throws Exception
+   */
+  public void clusterSetUp(Configuration hdfsConf) throws Exception {
+    clusterSetUp(false, hdfsConf);
   }
 
   /**
@@ -80,8 +95,9 @@ public class TestPersistentStoragePolicySatisfier {
    * @param isHAEnabled if true, enable simple HA.
    * @throws IOException
    */
-  private void clusterSetUp(boolean isHAEnabled) throws Exception {
-    conf = new HdfsConfiguration();
+  private void clusterSetUp(boolean isHAEnabled, Configuration newConf)
+      throws Exception {
+    conf = newConf;
     final int dnNumber = storageTypes.length;
     final short replication = 3;
     MiniDFSCluster.Builder clusterBuilder = new MiniDFSCluster.Builder(conf)
@@ -188,7 +204,7 @@ public class TestPersistentStoragePolicySatisfier {
   public void testWithHA() throws Exception {
     try {
       // Enable HA env for testing.
-      clusterSetUp(true);
+      clusterSetUp(true, new HdfsConfiguration());
 
       fs.setStoragePolicy(testFile, ALL_SSD);
       fs.satisfyStoragePolicy(testFile);
@@ -298,6 +314,94 @@ public class TestPersistentStoragePolicySatisfier {
   }
 
   /**
+   * Tests to verify SPS xattr will be removed if the satisfy work has
+   * been finished, expect that the method satisfyStoragePolicy can be
+   * invoked on the same file again after the block movement has been
+   * finished:
+   * 1. satisfy storage policy of file1.
+   * 2. wait until storage policy is satisfied.
+   * 3. satisfy storage policy of file1 again
+   * 4. make sure step 3 works as expected.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testMultipleSatisfyStoragePolicy() throws Exception {
+    try {
+      // Lower block movement check for testing.
+      conf = new HdfsConfiguration();
+      final long minCheckTimeout = 500; // minimum value
+      conf.setLong(
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          minCheckTimeout);
+      clusterSetUp(conf);
+      fs.setStoragePolicy(testFile, ONE_SSD);
+      fs.satisfyStoragePolicy(testFile);
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.SSD, 1, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.DISK, 2, timeout, fs);
+
+      // Make sure that SPS xattr has been removed.
+      int retryTime = 0;
+      while (retryTime < 30) {
+        if (!fileContainsSPSXAttr(testFile)) {
+          break;
+        }
+        Thread.sleep(minCheckTimeout);
+        retryTime += 1;
+      }
+
+      fs.setStoragePolicy(testFile, COLD);
+      fs.satisfyStoragePolicy(testFile);
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.ARCHIVE, 3, timeout, fs);
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
+   * Tests to verify SPS xattr is removed after SPS is dropped,
+   * expect that if the SPS is disabled/dropped, the SPS
+   * xattr should be removed accordingly:
+   * 1. satisfy storage policy of file1.
+   * 2. drop SPS thread in block manager.
+   * 3. make sure sps xattr is removed.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testDropSPS() throws Exception {
+    try {
+      clusterSetUp();
+      fs.setStoragePolicy(testFile, ONE_SSD);
+      fs.satisfyStoragePolicy(testFile);
+
+      cluster.getNamesystem().getBlockManager().deactivateSPS();
+
+      // Make sure satisfy xattr has been removed.
+      assertFalse(fileContainsSPSXAttr(testFile));
+
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
+   * Check whether file contains SPS xattr.
+   * @param fileName file name.
+   * @return true if file contains SPS xattr.
+   * @throws IOException
+   */
+  private boolean fileContainsSPSXAttr(Path fileName) throws IOException {
+    final INode inode = cluster.getNamesystem()
+        .getFSDirectory().getINode(fileName.toString());
+    final XAttr satisfyXAttr =
+        XAttrHelper.buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
+    List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
+    return existingXAttrs.contains(satisfyXAttr);
+  }
+
+  /**
    * Restart the hole env and trigger the DataNode's heart beats.
    * @throws Exception
    */


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


[47/50] [abbrv] hadoop git commit: HDFS-12556: [SPS] : Block movement analysis should be done in read lock.

Posted by ra...@apache.org.
HDFS-12556: [SPS] : Block movement analysis should be done in read lock.


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

Branch: refs/heads/HDFS-10285
Commit: e6e61b7762d5d8e991cf14918dee4d2633fe5155
Parents: 42b22d9
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Sat Oct 14 15:11:26 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:31:18 2017 +0530

----------------------------------------------------------------------
 .../server/namenode/StoragePolicySatisfier.java | 27 +++++++++++++-------
 .../TestPersistentStoragePolicySatisfier.java   |  2 +-
 2 files changed, 19 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6e61b77/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 a28a806..cbfba44 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
@@ -242,12 +242,25 @@ public class StoragePolicySatisfier implements Runnable {
           ItemInfo itemInfo = storageMovementNeeded.get();
           if (itemInfo != null) {
             long trackId = itemInfo.getTrackId();
-            BlockCollection blockCollection =
-                namesystem.getBlockCollection(trackId);
-            // Check blockCollectionId existence.
+            BlockCollection blockCollection;
+            BlocksMovingAnalysis status = null;
+            try {
+              namesystem.readLock();
+              blockCollection = namesystem.getBlockCollection(trackId);
+              // Check blockCollectionId existence.
+              if (blockCollection == null) {
+                // File doesn't exists (maybe got deleted), remove trackId from
+                // the queue
+                storageMovementNeeded.removeItemTrackInfo(itemInfo);
+              } else {
+                status =
+                    analyseBlocksStorageMovementsAndAssignToDN(
+                        blockCollection);
+              }
+            } finally {
+              namesystem.readUnlock();
+            }
             if (blockCollection != null) {
-              BlocksMovingAnalysis status =
-                  analyseBlocksStorageMovementsAndAssignToDN(blockCollection);
               switch (status.status) {
               // Just add to monitor, so it will be retried after timeout
               case ANALYSIS_SKIPPED_FOR_RETRY:
@@ -283,10 +296,6 @@ public class StoragePolicySatisfier implements Runnable {
                 storageMovementNeeded.removeItemTrackInfo(itemInfo);
                 break;
               }
-            } else {
-              // File doesn't exists (maybe got deleted), remove trackId from
-              // the queue
-              storageMovementNeeded.removeItemTrackInfo(itemInfo);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6e61b77/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 5bce296..7165d06 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
@@ -72,7 +72,7 @@ public class TestPersistentStoragePolicySatisfier {
       {StorageType.DISK, StorageType.ARCHIVE, StorageType.SSD}
   };
 
-  private final int timeout = 300000;
+  private final int timeout = 90000;
 
   /**
    * Setup environment for every test case.


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


[34/50] [abbrv] hadoop git commit: HDFS-11965: [SPS]: Should give chance to satisfy the low redundant blocks before removing the xattr. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-11965: [SPS]: Should give chance to satisfy the low redundant blocks before removing the xattr. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: fd26f718bbf3e8f42ed4875722e62e1aa6a3e211
Parents: 7191347
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jul 10 18:00:58 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:19 2017 +0530

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |  15 +++
 .../server/namenode/StoragePolicySatisfier.java |  20 +++-
 .../namenode/TestStoragePolicySatisfier.java    | 102 ++++++++++++++++++-
 ...stStoragePolicySatisfierWithStripedFile.java |  90 ++++++++++++++++
 4 files changed, 224 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd26f718/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 d1bb570..37d6c1b 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
@@ -4265,6 +4265,21 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
+   * Check file has low redundancy blocks.
+   */
+  public boolean hasLowRedundancyBlocks(BlockCollection bc) {
+    boolean result = false;
+    for (BlockInfo block : bc.getBlocks()) {
+      short expected = getExpectedRedundancyNum(block);
+      final NumberReplicas n = countNodes(block);
+      if (expected > n.liveReplicas()) {
+        result = true;
+      }
+    }
+    return result;
+  }
+
+  /**
    * Check sufficient redundancy of the blocks in the collection. If any block
    * is needed reconstruction, insert it into the reconstruction queue.
    * Otherwise, if the block is more than the expected replication factor,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd26f718/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 1b2afa3..97cbf1b 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
@@ -99,7 +99,10 @@ public class StoragePolicySatisfier implements Runnable {
     // Represents that, the analysis skipped due to some conditions.
     // Example conditions are if no blocks really exists in block collection or
     // if analysis is not required on ec files with unsuitable storage policies
-    BLOCKS_TARGET_PAIRING_SKIPPED;
+    BLOCKS_TARGET_PAIRING_SKIPPED,
+    // Represents that, All the reported blocks are satisfied the policy but
+    // some of the blocks are low redundant.
+    FEW_LOW_REDUNDANCY_BLOCKS
   }
 
   public StoragePolicySatisfier(final Namesystem namesystem,
@@ -247,6 +250,14 @@ public class StoragePolicySatisfier implements Runnable {
               case FEW_BLOCKS_TARGETS_PAIRED:
                 this.storageMovementsMonitor.add(blockCollectionID, false);
                 break;
+              case FEW_LOW_REDUNDANCY_BLOCKS:
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Adding trackID " + blockCollectionID
+                      + " back to retry queue as some of the blocks"
+                      + " are low redundant.");
+                }
+                this.storageMovementNeeded.add(blockCollectionID);
+                break;
               // Just clean Xattrs
               case BLOCKS_TARGET_PAIRING_SKIPPED:
               case BLOCKS_ALREADY_SATISFIED:
@@ -347,11 +358,16 @@ public class StoragePolicySatisfier implements Runnable {
         boolean computeStatus = computeBlockMovingInfos(blockMovingInfos,
             blockInfo, expectedStorageTypes, existing, storages);
         if (computeStatus
-            && status != BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED) {
+            && status != BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED
+            && !blockManager.hasLowRedundancyBlocks(blockCollection)) {
           status = BlocksMovingAnalysisStatus.ALL_BLOCKS_TARGETS_PAIRED;
         } else {
           status = BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED;
         }
+      } else {
+        if (blockManager.hasLowRedundancyBlocks(blockCollection)) {
+          status = BlocksMovingAnalysisStatus.FEW_LOW_REDUNDANCY_BLOCKS;
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd26f718/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 f1a4169..7127895 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 
 import java.io.FileNotFoundException;
@@ -29,6 +30,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -41,6 +43,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
@@ -55,6 +58,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -582,7 +586,9 @@ public class TestStoragePolicySatisfier {
       Assert.assertTrue("SPS should be running as "
           + "no Mover really running", running);
     } finally {
-      hdfsCluster.shutdown();
+      if (hdfsCluster != null) {
+        hdfsCluster.shutdown();
+      }
     }
   }
 
@@ -983,6 +989,100 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Test SPS for low redundant file blocks.
+   * 1. Create cluster with 3 datanode.
+   * 1. Create one file with 3 replica.
+   * 2. Set policy and call satisfyStoragePolicy for file.
+   * 3. Stop NameNode and Datanodes.
+   * 4. Start NameNode with 2 datanode and wait for block movement.
+   * 5. Start third datanode.
+   * 6. Third Datanode replica also should be moved in proper
+   * sorage based on policy.
+   */
+  @Test(timeout = 300000)
+  public void testSPSWhenFileHasLowRedundancyBlocks() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      conf.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "3000");
+      StorageType[][] newtypes = new StorageType[][] {
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK}};
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+          .storageTypes(newtypes).build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path filePath = new Path("/zeroSizeFile");
+      DFSTestUtil.createFile(fs, filePath, 1024, (short) 3, 0);
+      fs.setStoragePolicy(filePath, "COLD");
+      List<DataNodeProperties> list = new ArrayList<>();
+      list.add(cluster.stopDataNode(0));
+      list.add(cluster.stopDataNode(0));
+      list.add(cluster.stopDataNode(0));
+      cluster.restartNameNodes();
+      cluster.restartDataNode(list.get(0), true);
+      cluster.restartDataNode(list.get(1), true);
+      cluster.waitActive();
+      fs.satisfyStoragePolicy(filePath);
+      Thread.sleep(3000 * 6);
+      cluster.restartDataNode(list.get(2), true);
+      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+          StorageType.ARCHIVE, 3, 30000, cluster.getFileSystem());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test SPS for extra redundant file blocks.
+   * 1. Create cluster with 5 datanode.
+   * 2. Create one file with 5 replica.
+   * 3. Set file replication to 3.
+   * 4. Set policy and call satisfyStoragePolicy for file.
+   * 5. Block should be moved successfully.
+   */
+  @Test(timeout = 300000)
+  public void testSPSWhenFileHasExcessRedundancyBlocks() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      conf.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "3000");
+      StorageType[][] newtypes = new StorageType[][] {
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK}};
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5)
+          .storageTypes(newtypes).build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path filePath = new Path("/zeroSizeFile");
+      DFSTestUtil.createFile(fs, filePath, 1024, (short) 5, 0);
+      fs.setReplication(filePath, (short) 3);
+      LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(
+          LogFactory.getLog(BlockStorageMovementAttemptedItems.class));
+      fs.setStoragePolicy(filePath, "COLD");
+      fs.satisfyStoragePolicy(filePath);
+      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+          StorageType.ARCHIVE, 3, 30000, cluster.getFileSystem());
+      assertFalse("Log output does not contain expected log message: ",
+          logs.getOutput().contains("some of the blocks are low redundant"));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
       throws IOException {
     ArrayList<DataNode> dns = hdfsCluster.getDataNodes();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd26f718/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 eb4a6a3..195c9e3 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -27,8 +29,10 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
@@ -293,6 +297,92 @@ public class TestStoragePolicySatisfierWithStripedFile {
   }
 
   /**
+   * Test SPS for low redundant file blocks.
+   * 1. Create cluster with 10 datanode.
+   * 1. Create one striped file with default EC Policy.
+   * 2. Set policy and call satisfyStoragePolicy for file.
+   * 3. Stop NameNode and Datanodes.
+   * 4. Start NameNode with 5 datanode and wait for block movement.
+   * 5. Start remaining 5 datanode.
+   * 6. All replica  should be moved in proper storage based on policy.
+   */
+  @Test(timeout = 300000)
+  public void testSPSWhenFileHasLowRedundancyBlocks() throws Exception {
+    // start 10 datanodes
+    int numOfDatanodes = 10;
+    int storagesPerDatanode = 2;
+    long capacity = 20 * defaultStripeBlockSize;
+    long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
+    for (int i = 0; i < numOfDatanodes; i++) {
+      for (int j = 0; j < storagesPerDatanode; j++) {
+        capacities[i][j] = capacity;
+      }
+    }
+
+    final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys
+        .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+        "3000");
+    initConfWithStripe(conf, defaultStripeBlockSize);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numOfDatanodes)
+        .storagesPerDatanode(storagesPerDatanode)
+        .storageTypes(new StorageType[][]{
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE}})
+        .storageCapacities(capacities)
+        .build();
+    try {
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path barDir = new Path("/bar");
+      fs.mkdirs(barDir);
+      // set an EC policy on "/bar" directory
+      fs.setErasureCodingPolicy(barDir, null);
+
+      // write file to barDir
+      final Path fooFile = new Path("/bar/foo");
+      long fileLen = cellSize * dataBlocks;
+      DFSTestUtil.createFile(cluster.getFileSystem(), fooFile,
+          fileLen, (short) 3, 0);
+
+      // Move file to ARCHIVE.
+      fs.setStoragePolicy(barDir, "COLD");
+      //Stop DataNodes and restart namenode
+      List<DataNodeProperties> list = new ArrayList<>(numOfDatanodes);
+      for (int i = 0; i < numOfDatanodes; i++) {
+        list.add(cluster.stopDataNode(0));
+      }
+      cluster.restartNameNodes();
+      // Restart half datanodes
+      for (int i = 0; i < numOfDatanodes / 2; i++) {
+        cluster.restartDataNode(list.get(i), true);
+      }
+      cluster.waitActive();
+      fs.satisfyStoragePolicy(fooFile);
+      Thread.sleep(3000 * 6);
+      //Start reaming datanodes
+      for (int i = numOfDatanodes - 1; i > numOfDatanodes / 2; i--) {
+        cluster.restartDataNode(list.get(i), true);
+      }
+      // verify storage types and locations.
+      waitExpectedStorageType(cluster, fooFile.toString(), fileLen,
+          StorageType.ARCHIVE, 9, 9, 60000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+
+  /**
    * Tests to verify that for the given path, no blocks under the given path
    * will be scheduled for block movement as there are no available datanode
    * with required storage type.


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


[28/50] [abbrv] hadoop git commit: HDFS-11695: [SPS]: Namenode failed to start while loading SPS xAttrs from the edits log. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-11695: [SPS]: Namenode failed to start while loading SPS xAttrs from the edits log. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 76da9b13fbfb3bab8768f9717b53975cfa7e64ea
Parents: 24c819b
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon May 22 21:39:43 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:37 2017 +0530

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSDirAttrOp.java       |  91 ------------
 .../namenode/FSDirSatisfyStoragePolicyOp.java   | 145 +++++++++++++++++++
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   2 +-
 .../hdfs/server/namenode/FSDirectory.java       |  16 --
 .../hdfs/server/namenode/FSNamesystem.java      |  24 ++-
 .../hadoop/hdfs/server/namenode/Namesystem.java |  10 ++
 .../server/namenode/StoragePolicySatisfier.java |   4 +-
 .../TestPersistentStoragePolicySatisfier.java   |  90 +++++++++++-
 .../namenode/TestStoragePolicySatisfier.java    |   5 +-
 9 files changed, 268 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 0132f36..0dfaa8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -43,14 +42,12 @@ import com.google.common.collect.Lists;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 
 public class FSDirAttrOp {
   static FileStatus setPermission(
@@ -200,29 +197,6 @@ public class FSDirAttrOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  static FileStatus satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
-      String src, boolean logRetryCache) throws IOException {
-
-    FSPermissionChecker pc = fsd.getPermissionChecker();
-    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
-    INodesInPath iip;
-    fsd.writeLock();
-    try {
-
-      // check operation permission.
-      iip = fsd.resolvePath(pc, src, DirOp.WRITE);
-      if (fsd.isPermissionEnabled()) {
-        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
-      }
-      XAttr satisfyXAttr = unprotectedSatisfyStoragePolicy(iip, bm, fsd);
-      xAttrs.add(satisfyXAttr);
-    } finally {
-      fsd.writeUnlock();
-    }
-    fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
-    return fsd.getAuditFileInfo(iip);
-  }
-
   static BlockStoragePolicy[] getStoragePolicies(BlockManager bm)
       throws IOException {
     return bm.getStoragePolicies();
@@ -484,71 +458,6 @@ public class FSDirAttrOp {
     }
   }
 
-  static XAttr unprotectedSatisfyStoragePolicy(INodesInPath iip,
-      BlockManager bm, FSDirectory fsd) throws IOException {
-
-    final INode inode = FSDirectory.resolveLastINode(iip);
-    final int snapshotId = iip.getLatestSnapshotId();
-    final List<INode> candidateNodes = new ArrayList<>();
-
-    // TODO: think about optimization here, label the dir instead
-    // of the sub-files of the dir.
-    if (inode.isFile()) {
-      candidateNodes.add(inode);
-    } else if (inode.isDirectory()) {
-      for (INode node : inode.asDirectory().getChildrenList(snapshotId)) {
-        if (node.isFile()) {
-          candidateNodes.add(node);
-        }
-      }
-    }
-
-    // If node has satisfy xattr, then stop adding it
-    // to satisfy movement queue.
-    if (inodeHasSatisfyXAttr(candidateNodes)) {
-      throw new IOException(
-          "Cannot request to call satisfy storage policy on path "
-          + iip.getPath()
-          + ", as this file/dir was already called for satisfying "
-          + "storage policy.");
-    }
-
-    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
-    final XAttr satisfyXAttr =
-        XAttrHelper.buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
-    xattrs.add(satisfyXAttr);
-
-    for (INode node : candidateNodes) {
-      bm.satisfyStoragePolicy(node.getId());
-      List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(node);
-      List<XAttr> newXAttrs = FSDirXAttrOp.setINodeXAttrs(
-          fsd, existingXAttrs, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
-      XAttrStorage.updateINodeXAttrs(node, newXAttrs, snapshotId);
-    }
-    return satisfyXAttr;
-  }
-
-  private static boolean inodeHasSatisfyXAttr(List<INode> candidateNodes) {
-    // If the node is a directory and one of the child files
-    // has satisfy xattr, then return true for this directory.
-    for (INode inode : candidateNodes) {
-      final XAttrFeature f = inode.getXAttrFeature();
-      if (inode.isFile() &&
-          f != null && f.getXAttr(XATTR_SATISFY_STORAGE_POLICY) != null) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  static void unprotectedRemoveSPSXAttr(INode inode, XAttr spsXAttr)
-      throws IOException{
-    List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
-    existingXAttrs.remove(spsXAttr);
-    XAttrStorage.updateINodeXAttrs(inode, existingXAttrs,
-        INodesInPath.fromINode(inode).getLatestSnapshotId());
-  }
-
   private static void setDirStoragePolicy(
       FSDirectory fsd, INodesInPath iip, byte policyId) throws IOException {
     INode inode = FSDirectory.resolveLastINode(iip);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
new file mode 100644
index 0000000..81d337f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.XAttrHelper;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Helper class to perform storage policy satisfier related operations.
+ */
+final class FSDirSatisfyStoragePolicyOp {
+
+  /**
+   * Private constructor for preventing FSDirSatisfyStoragePolicyOp object
+   * creation. Static-only class.
+   */
+  private FSDirSatisfyStoragePolicyOp() {
+  }
+
+  static FileStatus satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
+      String src, boolean logRetryCache) throws IOException {
+
+    assert fsd.getFSNamesystem().hasWriteLock();
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+    INodesInPath iip;
+    fsd.writeLock();
+    try {
+
+      // check operation permission.
+      iip = fsd.resolvePath(pc, src, DirOp.WRITE);
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
+      }
+      XAttr satisfyXAttr = unprotectedSatisfyStoragePolicy(iip, bm, fsd);
+      xAttrs.add(satisfyXAttr);
+      fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+    } finally {
+      fsd.writeUnlock();
+    }
+    return fsd.getAuditFileInfo(iip);
+  }
+
+  static XAttr unprotectedSatisfyStoragePolicy(INodesInPath iip,
+      BlockManager bm, FSDirectory fsd) throws IOException {
+
+    final INode inode = FSDirectory.resolveLastINode(iip);
+    final int snapshotId = iip.getLatestSnapshotId();
+    final List<INode> candidateNodes = new ArrayList<>();
+
+    // TODO: think about optimization here, label the dir instead
+    // of the sub-files of the dir.
+    if (inode.isFile()) {
+      candidateNodes.add(inode);
+    } else if (inode.isDirectory()) {
+      for (INode node : inode.asDirectory().getChildrenList(snapshotId)) {
+        if (node.isFile()) {
+          candidateNodes.add(node);
+        }
+      }
+    }
+
+    // If node has satisfy xattr, then stop adding it
+    // to satisfy movement queue.
+    if (inodeHasSatisfyXAttr(candidateNodes)) {
+      throw new IOException(
+          "Cannot request to call satisfy storage policy on path "
+              + iip.getPath()
+              + ", as this file/dir was already called for satisfying "
+              + "storage policy.");
+    }
+
+    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+    final XAttr satisfyXAttr = XAttrHelper
+        .buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
+    xattrs.add(satisfyXAttr);
+
+    for (INode node : candidateNodes) {
+      bm.satisfyStoragePolicy(node.getId());
+      List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(node);
+      List<XAttr> newXAttrs = FSDirXAttrOp.setINodeXAttrs(fsd, existingXAttrs,
+          xattrs, EnumSet.of(XAttrSetFlag.CREATE));
+      XAttrStorage.updateINodeXAttrs(node, newXAttrs, snapshotId);
+    }
+    return satisfyXAttr;
+  }
+
+  private static boolean inodeHasSatisfyXAttr(List<INode> candidateNodes) {
+    // If the node is a directory and one of the child files
+    // has satisfy xattr, then return true for this directory.
+    for (INode inode : candidateNodes) {
+      final XAttrFeature f = inode.getXAttrFeature();
+      if (inode.isFile() && f != null
+          && f.getXAttr(XATTR_SATISFY_STORAGE_POLICY) != null) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  static void removeSPSXattr(FSDirectory fsd, INode inode, XAttr spsXAttr)
+      throws IOException {
+    try {
+      fsd.writeLock();
+      List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
+      existingXAttrs.remove(spsXAttr);
+      XAttrStorage.updateINodeXAttrs(inode, existingXAttrs, INodesInPath
+          .fromINode(inode).getLatestSnapshotId());
+      List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+      xAttrs.add(spsXAttr);
+      fsd.getEditLog().logRemoveXAttrs(inode.getFullPathName(), xAttrs, false);
+    } finally {
+      fsd.writeUnlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
index b0bda10..488cc9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
@@ -291,7 +291,7 @@ class FSDirXAttrOp {
 
       // Add inode id to movement queue if xattrs contain satisfy xattr.
       if (XATTR_SATISFY_STORAGE_POLICY.equals(xaName)) {
-        FSDirAttrOp.unprotectedSatisfyStoragePolicy(iip,
+        FSDirSatisfyStoragePolicyOp.unprotectedSatisfyStoragePolicy(iip,
             fsd.getBlockManager(), fsd);
         continue;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 89b8ec3..37ed027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -1416,22 +1416,6 @@ public class FSDirectory implements Closeable {
     getBlockManager().satisfyStoragePolicy(inode.getId());
   }
 
-  /**
-   * Remove the SPS xattr from the inode, retrieve the inode from the
-   * block collection id.
-   * @param id
-   *           - file block collection id.
-   */
-  public void removeSPSXattr(long id) throws IOException {
-    final INode inode = getInode(id);
-    final XAttrFeature xaf = inode.getXAttrFeature();
-    final XAttr spsXAttr = xaf.getXAttr(XATTR_SATISFY_STORAGE_POLICY);
-
-    if (spsXAttr != null) {
-      FSDirAttrOp.unprotectedRemoveSPSXAttr(inode, spsXAttr);
-    }
-  }
-
   private void addEncryptionZone(INodeWithAdditionalFields inode,
       XAttrFeature xaf) {
     if (xaf == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/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 55f3e84..c5ab56b 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
@@ -2191,10 +2191,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 + " by admin. Seek for an admin help to activate it "
                 + "or use Mover tool.");
       }
-      FSDirAttrOp.satisfyStoragePolicy(dir, blockManager, src, logRetryCache);
+      FSDirSatisfyStoragePolicyOp.satisfyStoragePolicy(dir, blockManager, src,
+          logRetryCache);
     } finally {
       writeUnlock();
     }
+    getEditLog().logSync();
   }
 
   /**
@@ -7679,6 +7681,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, operationName, src, null, auditStat);
   }
 
+  @Override
+  public void removeXattr(long id, String xattrName) throws IOException {
+    writeLock();
+    try {
+      final INode inode = dir.getInode(id);
+      final XAttrFeature xaf = inode.getXAttrFeature();
+      if (xaf == null) {
+        return;
+      }
+      final XAttr spsXAttr = xaf.getXAttr(xattrName);
+
+      if (spsXAttr != null) {
+        FSDirSatisfyStoragePolicyOp.removeSPSXattr(dir, inode, spsXAttr);
+      }
+    } finally {
+      writeUnlock("removeXAttr");
+    }
+    getEditLog().logSync();
+  }
+
   void checkAccess(String src, FsAction mode) throws IOException {
     final String operationName = "checkAccess";
     checkOperation(OperationCategory.READ);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index a2b07ca..e58fa72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
@@ -52,4 +54,12 @@ public interface Namesystem extends RwLock, SafeMode {
    * @return true if valid write lease exists, otherwise return false.
    */
   boolean isFileOpenedForWrite(String filePath);
+
+  /**
+   * Remove xAttr from the inode.
+   * @param id
+   * @param xattrName
+   * @throws IOException
+   */
+  void removeXattr(long id, String xattrName) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/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 3b20314..9e2a4a0 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -829,6 +831,6 @@ public class StoragePolicySatisfier implements Runnable {
    */
   public void postBlkStorageMovementCleanup(long trackId)
       throws IOException {
-    this.namesystem.getFSDirectory().removeSPSXattr(trackId);
+    this.namesystem.removeXattr(trackId, XATTR_SATISFY_STORAGE_POLICY);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/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 41c272c..bdf0159 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
@@ -32,6 +32,7 @@ import org.junit.Test;
 import java.io.IOException;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+import static org.junit.Assert.*;
 
 /**
  * Test persistence of satisfying files/directories.
@@ -60,11 +61,9 @@ public class TestPersistentStoragePolicySatisfier {
   private static final String ALL_SSD = "ALL_SSD";
 
   private static StorageType[][] storageTypes = new StorageType[][] {
-      {StorageType.ARCHIVE, StorageType.DISK},
-      {StorageType.DISK, StorageType.SSD},
-      {StorageType.SSD, StorageType.RAM_DISK},
-      {StorageType.ARCHIVE, StorageType.DISK},
-      {StorageType.ARCHIVE, StorageType.SSD}
+      {StorageType.DISK, StorageType.ARCHIVE, StorageType.SSD},
+      {StorageType.DISK, StorageType.ARCHIVE, StorageType.SSD},
+      {StorageType.DISK, StorageType.ARCHIVE, StorageType.SSD}
   };
 
   private final int timeout = 300000;
@@ -94,10 +93,13 @@ public class TestPersistentStoragePolicySatisfier {
   private void clusterSetUp(boolean isHAEnabled, Configuration newConf)
       throws Exception {
     conf = newConf;
+    conf.set(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+        "3000");
     final int dnNumber = storageTypes.length;
     final short replication = 3;
     MiniDFSCluster.Builder clusterBuilder = new MiniDFSCluster.Builder(conf)
-        .storageTypes(storageTypes)
+        .storageTypes(storageTypes).storagesPerDatanode(3)
         .numDataNodes(dnNumber);
     if (isHAEnabled) {
       clusterBuilder.nnTopology(MiniDFSNNTopology.simpleHATopology());
@@ -277,9 +279,10 @@ public class TestPersistentStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testWithFederationHA() throws Exception {
+    MiniDFSCluster haCluster = null;
     try {
       conf = new HdfsConfiguration();
-      final MiniDFSCluster haCluster = new MiniDFSCluster
+      haCluster = new MiniDFSCluster
           .Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
           .storageTypes(storageTypes)
@@ -305,7 +308,14 @@ public class TestPersistentStoragePolicySatisfier {
           testFileName, StorageType.ARCHIVE, 2, timeout, fs);
 
     } finally {
-      clusterShutdown();
+      if(fs != null) {
+        fs.close();
+        fs = null;
+      }
+      if(haCluster != null) {
+        haCluster.shutdown(true);
+        haCluster = null;
+      }
     }
   }
 
@@ -404,6 +414,70 @@ public class TestPersistentStoragePolicySatisfier {
   }
 
   /**
+   * Test loading of SPS xAttrs from the edits log when satisfyStoragePolicy
+   * called on child file and parent directory.
+   * 1. Create one directory and create one child file.
+   * 2. Set storage policy for child file and call
+   * satisfyStoragePolicy.
+   * 3. wait for SPS to remove xAttr for file child file.
+   * 4. Set storage policy for parent directory and call
+   * satisfyStoragePolicy.
+   * 5. restart the namenode.
+   * NameNode should be started successfully.
+   */
+  @Test(timeout = 300000)
+  public void testNameNodeRestartWhenSPSCalledOnChildFileAndParentDir()
+      throws Exception {
+    try {
+      clusterSetUp();
+      fs.setStoragePolicy(childFile, "COLD");
+      fs.satisfyStoragePolicy(childFile);
+      DFSTestUtil.waitExpectedStorageType(childFile.toUri().getPath(),
+          StorageType.ARCHIVE, 3, 30000, cluster.getFileSystem());
+      // wait for SPS to remove Xattr from file
+      Thread.sleep(30000);
+      fs.setStoragePolicy(childDir, "COLD");
+      fs.satisfyStoragePolicy(childDir);
+      try {
+        cluster.restartNameNodes();
+      } catch (Exception e) {
+        assertFalse(e.getMessage().contains(
+            "Cannot request to call satisfy storage policy"));
+      }
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
+   * Test SPS when satisfyStoragePolicy called on child file and
+   * parent directory.
+   * 1. Create one parent directory and child directory.
+   * 2. Create some file in both the directory.
+   * 3. Set storage policy for parent directory and call
+   * satisfyStoragePolicy.
+   * 4. Set storage policy for child directory and call
+   * satisfyStoragePolicy.
+   * 5. restart the namenode.
+   * All the file blocks should satisfy the policy.
+   */
+  @Test(timeout = 300000)
+  public void testSPSOnChildAndParentDirectory() throws Exception {
+    try {
+      clusterSetUp();
+      fs.setStoragePolicy(parentDir, "COLD");
+      fs.satisfyStoragePolicy(childDir);
+      fs.satisfyStoragePolicy(parentDir);
+      DFSTestUtil.waitExpectedStorageType(childFileName, StorageType.ARCHIVE,
+          3, 30000, cluster.getFileSystem());
+      DFSTestUtil.waitExpectedStorageType(parentFileName, StorageType.ARCHIVE,
+          3, 30000, cluster.getFileSystem());
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
    * Restart the hole env and trigger the DataNode's heart beats.
    * @throws Exception
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76da9b13/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 8457e5b..fa954b8 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -855,7 +856,9 @@ public class TestStoragePolicySatisfier {
             {StorageType.DISK, StorageType.SSD}};
 
     int defaultStripedBlockSize =
-        ErasureCodingPolicyManager.getSystemPolicies()[0].getCellSize() * 4;
+        StripedFileTestUtil.getDefaultECPolicy().getCellSize() * 4;
+    config.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        StripedFileTestUtil.getDefaultECPolicy().getName());
     config.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultStripedBlockSize);
     config.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
     config.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,


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


[11/50] [abbrv] hadoop git commit: HDFS-11151. [SPS]: StoragePolicySatisfier should gracefully handle when there is no target node with the required storage type. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-11151. [SPS]: StoragePolicySatisfier should gracefully handle when there is no target node with the required storage type. 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/a9adc9d7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a9adc9d7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a9adc9d7

Branch: refs/heads/HDFS-10285
Commit: a9adc9d7fbdb562b720c1123694208cbe6f792a7
Parents: 4d2d053
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Sun Nov 27 11:15:26 2016 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:20:35 2017 +0530

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |   2 +-
 .../datanode/BlockStorageMovementTracker.java   |  30 ++++--
 .../datanode/StoragePolicySatisfyWorker.java    |  20 +++-
 .../BlockStorageMovementAttemptedItems.java     |   4 +
 .../server/namenode/StoragePolicySatisfier.java |  53 ++++++---
 .../namenode/TestStoragePolicySatisfier.java    | 108 ++++++++++++++++++-
 6 files changed, 186 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9adc9d7/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 a5edbce..a0fe450 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
@@ -4921,7 +4921,7 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public void satisfyStoragePolicy(long id) {
     storageMovementNeeded.add(id);
-    if(LOG.isDebugEnabled()) {
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Added block collection id {} to block "
           + "storageMovementNeeded queue", id);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9adc9d7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
index d31f075..2de88fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -27,8 +27,9 @@ import java.util.concurrent.Future;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlocksMovementsCompletionHandler;
 import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementResult;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementStatus;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlocksMovementsCompletionHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -108,15 +109,32 @@ public class BlockStorageMovementTracker implements Runnable {
     }
   }
 
+  /**
+   * Mark as block movement failure for the given trackId and blockId.
+   *
+   * @param trackId tracking id
+   * @param blockId block id
+   */
+  void markBlockMovementFailure(long trackId, long blockId) {
+    LOG.debug("Mark as block movement failure for the given "
+        + "trackId:{} and blockId:{}", trackId, blockId);
+    BlockMovementResult result = new BlockMovementResult(trackId, blockId, null,
+        BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE);
+    addMovementResultToTrackIdList(result);
+  }
+
   private List<BlockMovementResult> addMovementResultToTrackIdList(
       BlockMovementResult result) {
     long trackId = result.getTrackId();
-    List<BlockMovementResult> perTrackIdList = movementResults.get(trackId);
-    if (perTrackIdList == null) {
-      perTrackIdList = new ArrayList<>();
-      movementResults.put(trackId, perTrackIdList);
+    List<BlockMovementResult> perTrackIdList;
+    synchronized (movementResults) {
+      perTrackIdList = movementResults.get(trackId);
+      if (perTrackIdList == null) {
+        perTrackIdList = new ArrayList<>();
+        movementResults.put(trackId, perTrackIdList);
+      }
+      perTrackIdList.add(result);
     }
-    perTrackIdList.add(result);
     return perTrackIdList;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9adc9d7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 1bd851e..a69a38b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -151,14 +151,24 @@ public class StoragePolicySatisfyWorker {
    */
   public void processBlockMovingTasks(long trackID, String blockPoolID,
       Collection<BlockMovingInfo> blockMovingInfos) {
+    LOG.debug("Received BlockMovingTasks {}", blockMovingInfos);
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      assert blkMovingInfo
-          .getSources().length == blkMovingInfo.getTargets().length;
-
-      for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
+      // Iterating backwards. This is to ensure that all the block src location
+      // which doesn't have a target node will be marked as failure before
+      // scheduling the block movement to valid target nodes.
+      for (int i = blkMovingInfo.getSources().length - 1; i >= 0; i--) {
+        if (i >= blkMovingInfo.getTargets().length) {
+          // Since there is no target selected for scheduling the block,
+          // just mark this block storage movement as failure. Later, namenode
+          // can take action on this.
+          movementTracker.markBlockMovementFailure(trackID,
+              blkMovingInfo.getBlock().getBlockId());
+          continue;
+        }
+        DatanodeInfo target = blkMovingInfo.getTargets()[i];
         BlockMovingTask blockMovingTask = new BlockMovingTask(
             trackID, blockPoolID, blkMovingInfo.getBlock(),
-            blkMovingInfo.getSources()[i], blkMovingInfo.getTargets()[i],
+            blkMovingInfo.getSources()[i], target,
             blkMovingInfo.getSourceStorageTypes()[i],
             blkMovingInfo.getTargetStorageTypes()[i]);
         Future<BlockMovementResult> moveCallable = moverCompletionService

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9adc9d7/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 580d0d6..5457dc2 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
@@ -218,4 +218,8 @@ public class BlockStorageMovementAttemptedItems {
     return storageMovementAttemptedResults.size();
   }
 
+  @VisibleForTesting
+  public int getAttemptedItemsCount() {
+    return storageMovementAttemptedItems.size();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9adc9d7/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 4967a89..617ab2c 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
@@ -211,6 +211,14 @@ public class StoragePolicySatisfier implements Runnable {
       }
     }
 
+    addBlockMovingInfosToCoordinatorDn(blockCollectionID, blockMovingInfos,
+        coordinatorNode);
+  }
+
+  private void addBlockMovingInfosToCoordinatorDn(long blockCollectionID,
+      List<BlockMovingInfo> blockMovingInfos,
+      DatanodeDescriptor coordinatorNode) {
+
     if (blockMovingInfos.size() < 1) {
       // TODO: Major: handle this case. I think we need retry cases to
       // be implemented. Idea is, if some files are not getting storage movement
@@ -218,6 +226,20 @@ public class StoragePolicySatisfier implements Runnable {
       return;
     }
 
+    boolean needBlockStorageMovement = false;
+    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
+      // Check for atleast one block storage movement has been chosen
+      if (blkMovingInfo.getTargets().length > 0){
+        needBlockStorageMovement = true;
+        break;
+      }
+    }
+    if (!needBlockStorageMovement) {
+      // Simply return as there is no targets selected for scheduling the block
+      // movement.
+      return;
+    }
+
     // 'BlockCollectionId' is used as the tracking ID. All the blocks under this
     // blockCollectionID will be added to this datanode.
     coordinatorNode.addBlocksToMoveStorage(blockCollectionID, blockMovingInfos);
@@ -251,9 +273,8 @@ public class StoragePolicySatisfier implements Runnable {
     List<DatanodeDescriptor> chosenNodes = new ArrayList<>();
     for (int i = 0; i < sourceWithStorageList.size(); i++) {
       StorageTypeNodePair existingTypeNodePair = sourceWithStorageList.get(i);
-      StorageTypeNodePair chosenTarget =
-          chooseTargetTypeInSameNode(existingTypeNodePair.dn, expected,
-              locsForExpectedStorageTypes, chosenNodes);
+      StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(
+          existingTypeNodePair.dn, expected);
 
       if (chosenTarget == null && blockManager.getDatanodeManager()
           .getNetworkTopology().isNodeGroupAware()) {
@@ -282,15 +303,14 @@ public class StoragePolicySatisfier implements Runnable {
         chosenNodes.add(chosenTarget.dn);
         // TODO: We can increment scheduled block count for this node?
       } else {
-        // TODO: Failed to ChooseTargetNodes...So let just retry. Shall we
-        // proceed without this targets? Then what should be final result?
-        // How about pack empty target, means target node could not be chosen ,
-        // so result should be RETRY_REQUIRED from DN always.
-        // Log..unable to choose target node for source datanodeDescriptor
+        LOG.warn(
+            "Failed to choose target datanode for the required"
+                + " storage types {}, block:{}, existing storage type:{}",
+            expected, blockInfo, existingTypeNodePair.storageType);
         sourceNodes.add(existingTypeNodePair.dn);
         sourceStorageTypes.add(existingTypeNodePair.storageType);
-        targetNodes.add(null);
-        targetStorageTypes.add(null);
+        // Imp: Not setting the target details, empty targets. Later, this is
+        // used as an indicator for retrying this block movement.
       }
     }
     BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blockInfo,
@@ -302,15 +322,13 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   /**
-   * Choose the target storage within same Datanode if possible.
+   * Choose the target storage within same datanode if possible.
    *
-   * @param locsForExpectedStorageTypes
-   * @param chosenNodes
+   * @param source source datanode
+   * @param targetTypes list of target storage types
    */
   private StorageTypeNodePair chooseTargetTypeInSameNode(
-      DatanodeDescriptor source, List<StorageType> targetTypes,
-      StorageTypeNodeMap locsForExpectedStorageTypes,
-      List<DatanodeDescriptor> chosenNodes) {
+      DatanodeDescriptor source, List<StorageType> targetTypes) {
     for (StorageType t : targetTypes) {
       DatanodeStorageInfo chooseStorage4Block =
           source.chooseStorage4Block(t, 0);
@@ -328,6 +346,9 @@ public class StoragePolicySatisfier implements Runnable {
     for (StorageType t : targetTypes) {
       List<DatanodeDescriptor> nodesWithStorages =
           locsForExpectedStorageTypes.getNodesWithStorages(t);
+      if (nodesWithStorages == null || nodesWithStorages.isEmpty()) {
+        continue; // no target nodes with the required storage type.
+      }
       Collections.shuffle(nodesWithStorages);
       for (DatanodeDescriptor target : nodesWithStorages) {
         if (!chosenNodes.contains(target) && matcher.match(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9adc9d7/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 901e1ba..499fe3c 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -44,8 +46,6 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Supplier;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
-
 /**
  * Tests that StoragePolicySatisfier daemon is able to check the blocks to be
  * moved and finding its suggested target locations to move.
@@ -79,7 +79,7 @@ public class TestStoragePolicySatisfier {
       throws Exception {
 
     try {
-      // Change policy to ALL_SSD
+      // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), "COLD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
@@ -356,6 +356,108 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Tests to verify that for the given path, some of the blocks or block src
+   * locations(src nodes) under the given path will be scheduled for block
+   * movement.
+   *
+   * For example, there are two block for a file:
+   *
+   * File1 => blk_1[locations=A(DISK),B(DISK),C(DISK)],
+   * blk_2[locations=A(DISK),B(DISK),C(DISK)]. Now, set storage policy to COLD.
+   * Only one datanode is available with storage type ARCHIVE, say D.
+   *
+   * SPS will schedule block movement to the coordinator node with the details,
+   * blk_1[move A(DISK) -> D(ARCHIVE)], blk_2[move A(DISK) -> D(ARCHIVE)].
+   */
+  @Test(timeout = 300000)
+  public void testWhenOnlyFewTargetDatanodeAreAvailableToSatisfyStoragePolicy()
+      throws Exception {
+    try {
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file), "COLD");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE}};
+
+      // Adding ARCHIVE based datanodes.
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier identified that block to move to
+      // ARCHIVE area.
+      waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
+
+      waitForBlocksMovementResult(1, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests to verify that for the given path, no blocks or block src
+   * locations(src nodes) under the given path will be scheduled for block
+   * movement as there are no available datanode with required storage type.
+   *
+   * For example, there are two block for a file:
+   *
+   * File1 => blk_1[locations=A(DISK),B(DISK),C(DISK)],
+   * blk_2[locations=A(DISK),B(DISK),C(DISK)]. Now, set storage policy to COLD.
+   * No datanode is available with storage type ARCHIVE.
+   *
+   * SPS won't schedule any block movement for this path.
+   */
+  @Test(timeout = 300000)
+  public void testWhenNoTargetDatanodeToSatisfyStoragePolicy()
+      throws Exception {
+    try {
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file), "COLD");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.DISK, StorageType.DISK}};
+      // Adding DISK based datanodes
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+
+      // No block movement will be scheduled as there is no target node available
+      // with the required storage type.
+      waitForAttemptedItems(1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, 3, 30000);
+      // Since there is no target node the item will get timed out and then
+      // re-attempted.
+      waitForAttemptedItems(1, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  private void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
+      int timeout) throws TimeoutException, InterruptedException {
+    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
+    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
+            expectedBlkMovAttemptedCount,
+            sps.getAttemptedItemsMonitor().getAttemptedItemsCount());
+        return sps.getAttemptedItemsMonitor()
+            .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
+      }
+    }, 100, timeout);
+  }
+
   private void waitForBlocksMovementResult(long expectedBlkMovResultsCount,
       int timeout) throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();


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


[18/50] [abbrv] hadoop git commit: HDFS-11293: [SPS]: Local DN should be given preference as source node, when target available in same node. Contributed by Yuanbo Liu and Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11293: [SPS]: Local DN should be given preference as source node, when target available in same node. Contributed by Yuanbo Liu and Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: 8dcd716c948578dbef87ff0a3867e7319c6e109b
Parents: aa6bf50
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jan 9 14:37:42 2017 -0800
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:45 2017 +0530

----------------------------------------------------------------------
 .../server/namenode/StoragePolicySatisfier.java | 49 ++++++++++++--
 .../namenode/TestStoragePolicySatisfier.java    | 71 ++++++++++++++++++++
 2 files changed, 113 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8dcd716c/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 ee59617..b1b1464 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
@@ -298,9 +298,25 @@ public class StoragePolicySatisfier implements Runnable {
           new ArrayList<StorageTypeNodePair>();
       List<DatanodeStorageInfo> existingBlockStorages =
           new ArrayList<DatanodeStorageInfo>(Arrays.asList(storages));
+      // if expected type exists in source node already, local movement would be
+      // possible, so lets find such sources first.
+      Iterator<DatanodeStorageInfo> iterator = existingBlockStorages.iterator();
+      while (iterator.hasNext()) {
+        DatanodeStorageInfo datanodeStorageInfo = iterator.next();
+        if (checkSourceAndTargetTypeExists(
+            datanodeStorageInfo.getDatanodeDescriptor(), existing,
+            expectedStorageTypes)) {
+          sourceWithStorageMap
+              .add(new StorageTypeNodePair(datanodeStorageInfo.getStorageType(),
+                  datanodeStorageInfo.getDatanodeDescriptor()));
+          iterator.remove();
+          existing.remove(datanodeStorageInfo.getStorageType());
+        }
+      }
+
+      // Let's find sources for existing types left.
       for (StorageType existingType : existing) {
-        Iterator<DatanodeStorageInfo> iterator =
-            existingBlockStorages.iterator();
+        iterator = existingBlockStorages.iterator();
         while (iterator.hasNext()) {
           DatanodeStorageInfo datanodeStorageInfo = iterator.next();
           StorageType storageType = datanodeStorageInfo.getStorageType();
@@ -317,7 +333,7 @@ public class StoragePolicySatisfier implements Runnable {
           findTargetsForExpectedStorageTypes(expectedStorageTypes);
 
       foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
-          blockMovingInfos, blockInfo, existing, sourceWithStorageMap,
+          blockMovingInfos, blockInfo, sourceWithStorageMap,
           expectedStorageTypes, locsForExpectedStorageTypes);
     }
     return foundMatchingTargetNodesForBlock;
@@ -366,8 +382,6 @@ public class StoragePolicySatisfier implements Runnable {
    *          - list of block source and target node pair
    * @param blockInfo
    *          - Block
-   * @param existing
-   *          - Existing storage types of block
    * @param sourceWithStorageList
    *          - Source Datanode with storages list
    * @param expected
@@ -379,7 +393,6 @@ public class StoragePolicySatisfier implements Runnable {
    */
   private boolean findSourceAndTargetToMove(
       List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
-      List<StorageType> existing,
       List<StorageTypeNodePair> sourceWithStorageList,
       List<StorageType> expected,
       StorageTypeNodeMap locsForExpectedStorageTypes) {
@@ -403,6 +416,7 @@ public class StoragePolicySatisfier implements Runnable {
         targetNodes.add(chosenTarget.dn);
         targetStorageTypes.add(chosenTarget.storageType);
         chosenNodes.add(chosenTarget.dn);
+        expected.remove(chosenTarget.storageType);
         // TODO: We can increment scheduled block count for this node?
       }
     }
@@ -442,16 +456,20 @@ public class StoragePolicySatisfier implements Runnable {
         targetNodes.add(chosenTarget.dn);
         targetStorageTypes.add(chosenTarget.storageType);
         chosenNodes.add(chosenTarget.dn);
+        expected.remove(chosenTarget.storageType);
         // TODO: We can increment scheduled block count for this node?
       } else {
         LOG.warn(
             "Failed to choose target datanode for the required"
                 + " storage types {}, block:{}, existing storage type:{}",
             expected, blockInfo, existingTypeNodePair.storageType);
-        foundMatchingTargetNodesForBlock = false;
       }
     }
 
+    if (expected.size() > 0) {
+      foundMatchingTargetNodesForBlock = false;
+    }
+
     blockMovingInfos.addAll(getBlockMovingInfos(blockInfo, sourceNodes,
         sourceStorageTypes, targetNodes, targetStorageTypes));
     return foundMatchingTargetNodesForBlock;
@@ -616,6 +634,23 @@ public class StoragePolicySatisfier implements Runnable {
     return max;
   }
 
+  private boolean checkSourceAndTargetTypeExists(DatanodeDescriptor dn,
+      List<StorageType> existing, List<StorageType> expectedStorageTypes) {
+    DatanodeStorageInfo[] allDNStorageInfos = dn.getStorageInfos();
+    boolean isExpectedTypeAvailable = false;
+    boolean isExistingTypeAvailable = false;
+    for (DatanodeStorageInfo dnInfo : allDNStorageInfos) {
+      StorageType storageType = dnInfo.getStorageType();
+      if (existing.contains(storageType)) {
+        isExistingTypeAvailable = true;
+      }
+      if (expectedStorageTypes.contains(storageType)) {
+        isExpectedTypeAvailable = true;
+      }
+    }
+    return isExistingTypeAvailable && isExpectedTypeAvailable;
+  }
+
   private static class StorageTypeNodeMap {
     private final EnumMap<StorageType, List<DatanodeDescriptor>> typeNodeMap =
         new EnumMap<StorageType, List<DatanodeDescriptor>>(StorageType.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8dcd716c/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 718dbcb..9abb78d 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
@@ -580,6 +580,77 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Tests that moving block storage with in the same datanode. Let's say we
+   * have DN1[DISK,ARCHIVE], DN2[DISK, SSD], DN3[DISK,RAM_DISK] when
+   * storagepolicy set to ONE_SSD and request satisfyStoragePolicy, then block
+   * should move to DN2[SSD] successfully.
+   */
+  @Test(timeout = 300000)
+  public void testBlockMoveInSameDatanodeWithONESSD() throws Exception {
+    StorageType[][] diskTypes =
+        new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.SSD},
+            {StorageType.DISK, StorageType.RAM_DISK}};
+    config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    try {
+      hdfsCluster = startCluster(config, diskTypes, numOfDatanodes,
+          storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(file);
+
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(file), "ONE_SSD");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+      waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
+
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Tests that moving block storage with in the same datanode and remote node.
+   * Let's say we have DN1[DISK,ARCHIVE], DN2[ARCHIVE, SSD], DN3[DISK,DISK],
+   * DN4[DISK,DISK] when storagepolicy set to WARM and request
+   * satisfyStoragePolicy, then block should move to DN1[ARCHIVE] and
+   * DN2[ARCHIVE] successfully.
+   */
+  @Test(timeout = 300000)
+  public void testBlockMoveInSameAndRemoteDatanodesWithWARM() throws Exception {
+    StorageType[][] diskTypes =
+        new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.ARCHIVE, StorageType.SSD},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK}};
+
+    config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    try {
+      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+          storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(file);
+
+      // Change policy to WARM
+      dfs.setStoragePolicy(new Path(file), "WARM");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+
+      waitExpectedStorageType(file, StorageType.DISK, 1, 30000);
+      waitExpectedStorageType(file, StorageType.ARCHIVE, 2, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
       throws IOException {
     ArrayList<DataNode> dns = hdfsCluster.getDataNodes();


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


[23/50] [abbrv] hadoop git commit: HDFS-11239: [SPS]: Check Mover file ID lease also to determine whether Mover is running. Contributed by Wei Zhou

Posted by ra...@apache.org.
HDFS-11239: [SPS]: Check Mover file ID lease also to determine whether Mover is running. Contributed by Wei Zhou


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

Branch: refs/heads/HDFS-10285
Commit: f8efef532591f2732f3798a39d4f86f2a9f93c27
Parents: 18d9bc5
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Feb 17 20:49:38 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:12 2017 +0530

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      |  17 ++-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   7 ++
 .../server/namenode/StoragePolicySatisfier.java |  19 +---
 .../namenode/TestStoragePolicySatisfier.java    | 108 +++++++++++++++----
 4 files changed, 113 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8efef53/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 2478466..0be96f3 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
@@ -3460,7 +3460,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   BlockInfo getStoredBlock(Block block) {
     return blockManager.getStoredBlock(block);
   }
-  
+
+  @Override
+  public boolean isFileOpenedForWrite(String path) {
+    readLock();
+    try {
+      INode inode = dir.getINode(path, FSDirectory.DirOp.READ);
+      INodeFile iNodeFile = INodeFile.valueOf(inode, path);
+      LeaseManager.Lease lease = leaseManager.getLease(iNodeFile);
+      return lease != null;
+    } catch (IOException e) {
+      return false;
+    } finally {
+      readUnlock();
+    }
+  }
+
   @Override
   public boolean isInSnapshot(long blockCollectionID) {
     assert hasReadLock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8efef53/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index e07376b..a2b07ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -45,4 +45,11 @@ public interface Namesystem extends RwLock, SafeMode {
    *         middle of the starting active services.
    */
   boolean inTransitionToActive();
+
+  /**
+   * Check if file is been opened for write purpose.
+   * @param filePath
+   * @return true if valid write lease exists, otherwise return false.
+   */
+  boolean isFileOpenedForWrite(String filePath);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8efef53/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 dc58294..29c8a5d 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
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -151,19 +150,8 @@ public class StoragePolicySatisfier implements Runnable {
 
   // Return true if a Mover instance is running
   private boolean checkIfMoverRunning() {
-    boolean ret = false;
-    try {
-      String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
-      INode inode = namesystem.getFSDirectory().getINode(
-          moverId, FSDirectory.DirOp.READ);
-      if (inode != null) {
-        ret = true;
-      }
-    } catch (IOException e) {
-      LOG.info("StoragePolicySatisfier is enabled as no Mover ID file found.");
-      ret = false;
-    }
-    return ret;
+    String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
+    return namesystem.isFileOpenedForWrite(moverId);
   }
 
   @Override
@@ -177,7 +165,8 @@ public class StoragePolicySatisfier implements Runnable {
         this.storageMovementsMonitor.stop();
         LOG.error(
             "Stopping StoragePolicySatisfier thread " + "as Mover ID file "
-                + HdfsServerConstants.MOVER_ID_PATH.toString() + " exists");
+                + HdfsServerConstants.MOVER_ID_PATH.toString()
+                + " been opened. Maybe a Mover instance is running!");
         return;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f8efef53/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 de73e8b..2a33455 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
@@ -29,6 +29,7 @@ import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -97,29 +98,33 @@ public class TestStoragePolicySatisfier {
 
     try {
       createCluster();
-      // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), COLD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
-
-      StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
-              {StorageType.ARCHIVE, StorageType.ARCHIVE},
-              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
-      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
-          storagesPerDatanode, capacity, hdfsCluster);
-
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
-
-      hdfsCluster.triggerHeartbeats();
-      // Wait till namenode notified about the block location details
-      DFSTestUtil.waitExpectedStorageType(
-          file, StorageType.ARCHIVE, 3, 30000, dfs);
+      doTestWhenStoragePolicySetToCOLD();
     } finally {
       shutdownCluster();
     }
   }
 
+  private void doTestWhenStoragePolicySetToCOLD() throws Exception {
+    // Change policy to COLD
+    dfs.setStoragePolicy(new Path(file), COLD);
+    FSNamesystem namesystem = hdfsCluster.getNamesystem();
+    INode inode = namesystem.getFSDirectory().getINode(file);
+
+    StorageType[][] newtypes =
+        new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
+            {StorageType.ARCHIVE, StorageType.ARCHIVE},
+            {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+    startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+        storagesPerDatanode, capacity, hdfsCluster);
+
+    namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+
+    hdfsCluster.triggerHeartbeats();
+    // Wait till namenode notified about the block location details
+    DFSTestUtil.waitExpectedStorageType(
+        file, StorageType.ARCHIVE, 3, 30000, dfs);
+  }
+
   @Test(timeout = 300000)
   public void testWhenStoragePolicySetToALLSSD()
       throws Exception {
@@ -500,19 +505,78 @@ public class TestStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
+      throws Exception {
+    boolean running;
+    FSDataOutputStream out = null;
+    try {
+      createCluster();
+      // Stop SPS
+      hdfsCluster.getNameNode().reconfigurePropertyImpl(
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "false");
+      running = hdfsCluster.getFileSystem()
+          .getClient().isStoragePolicySatisfierRunning();
+      Assert.assertFalse("SPS should stopped as configured.", running);
+
+      // Simulate the case by creating MOVER_ID file
+      out = hdfsCluster.getFileSystem().create(
+          HdfsServerConstants.MOVER_ID_PATH);
+
+      // Restart SPS
+      hdfsCluster.getNameNode().reconfigurePropertyImpl(
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "true");
+
+      running = hdfsCluster.getFileSystem()
+          .getClient().isStoragePolicySatisfierRunning();
+      Assert.assertFalse("SPS should not be able to run as file "
+          + HdfsServerConstants.MOVER_ID_PATH + " is being hold.", running);
+
+      // Simulate Mover exists
+      out.close();
+      out = null;
+      hdfsCluster.getFileSystem().delete(
+          HdfsServerConstants.MOVER_ID_PATH, true);
+
+      // Restart SPS again
+      hdfsCluster.getNameNode().reconfigurePropertyImpl(
+          DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "true");
+      running = hdfsCluster.getFileSystem()
+          .getClient().isStoragePolicySatisfierRunning();
+      Assert.assertTrue("SPS should be running as "
+          + "Mover already exited", running);
+
+      // Check functionality after SPS restart
+      doTestWhenStoragePolicySetToCOLD();
+    } catch (ReconfigurationException e) {
+      throw new IOException("Exception when reconfigure "
+          + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, e);
+    } finally {
+      if (out != null) {
+        out.close();
+      }
+      hdfsCluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests to verify that SPS should be able to start when the Mover ID file
+   * is not being hold by a Mover. This can be the case when Mover exits
+   * ungracefully without deleting the ID file from HDFS.
+   */
+  @Test(timeout = 300000)
+  public void testWhenMoverExitsWithoutDeleteMoverIDFile()
       throws IOException {
     try {
       createCluster();
-      // Simulate Mover by creating MOVER_ID file
+      // Simulate the case by creating MOVER_ID file
       DFSTestUtil.createFile(hdfsCluster.getFileSystem(),
           HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
       hdfsCluster.restartNameNode(true);
       boolean running = hdfsCluster.getFileSystem()
           .getClient().isStoragePolicySatisfierRunning();
-      Assert.assertFalse("SPS should not start "
-          + "when a Mover instance is running", running);
+      Assert.assertTrue("SPS should be running as "
+          + "no Mover really running", running);
     } finally {
-      shutdownCluster();
+      hdfsCluster.shutdown();
     }
   }
 


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


[08/50] [abbrv] hadoop git commit: HDFS-11029. [SPS]:Provide retry mechanism for the blocks which were failed while moving its storage at DNs. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11029. [SPS]:Provide retry mechanism for the blocks which were failed while moving its storage at DNs. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: 9ddfbe11424ca5a21dfc9dc6f67f0ef4e4d251e9
Parents: 83badc7
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Thu Nov 10 10:09:45 2016 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:20:17 2017 +0530

----------------------------------------------------------------------
 .../BlockStorageMovementAttemptedItems.java     | 221 +++++++++++++++++++
 .../server/namenode/StoragePolicySatisfier.java |  24 +-
 .../TestBlockStorageMovementAttemptedItems.java | 101 +++++++++
 .../namenode/TestStoragePolicySatisfier.java    |   8 +-
 4 files changed, 343 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ddfbe11/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
new file mode 100644
index 0000000..580d0d6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementAttemptedItems.java
@@ -0,0 +1,221 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.apache.hadoop.util.Time.monotonicNow;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * A monitor class for checking whether block storage movements finished or not.
+ * If block storage movement results from datanode indicates about the movement
+ * success, then it will just remove the entries from tracking. If it reports
+ * failure, then it will add back to needed block storage movements list. If no
+ * DN reports about movement for longer time, then such items will be retries
+ * automatically after timeout. The default timeout would be 30mins.
+ */
+public class BlockStorageMovementAttemptedItems {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
+  // A map holds the items which are already taken for blocks movements
+  // processing and sent to DNs.
+  private final Map<Long, Long> storageMovementAttemptedItems;
+  private final List<BlocksStorageMovementResult> storageMovementAttemptedResults;
+  private volatile boolean spsRunning = true;
+  private Daemon timerThread = null;
+  //
+  // It might take anywhere between 30 to 60 minutes before
+  // a request is timed out.
+  //
+  private long selfRetryTimeout = 30 * 60 * 1000;
+
+  //
+  // It might take anywhere between 5 to 10 minutes before
+  // a request is timed out.
+  //
+  private long checkTimeout = 5 * 60 * 1000; // minimum value
+  private BlockStorageMovementNeeded blockStorageMovementNeeded;
+
+  public BlockStorageMovementAttemptedItems(long timeoutPeriod,
+      long selfRetryTimeout,
+      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
+    if (timeoutPeriod > 0) {
+      this.checkTimeout = Math.min(checkTimeout, timeoutPeriod);
+    }
+
+    this.selfRetryTimeout = selfRetryTimeout;
+    this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
+    storageMovementAttemptedItems = new HashMap<>();
+    storageMovementAttemptedResults = new ArrayList<>();
+  }
+
+  /**
+   * Add item to block storage movement attempted items map which holds the
+   * tracking/blockCollection id versus time stamp.
+   *
+   * @param blockCollectionID
+   *          - tracking id / block collection id
+   */
+  public void add(Long blockCollectionID) {
+    synchronized (storageMovementAttemptedItems) {
+      storageMovementAttemptedItems.put(blockCollectionID, monotonicNow());
+    }
+  }
+
+  /**
+   * Add the trackIDBlocksStorageMovementResults to
+   * storageMovementAttemptedResults.
+   *
+   * @param blksMovementResults
+   */
+  public void addResults(BlocksStorageMovementResult[] blksMovementResults) {
+    if (blksMovementResults.length == 0) {
+      return;
+    }
+    synchronized (storageMovementAttemptedResults) {
+      storageMovementAttemptedResults
+          .addAll(Arrays.asList(blksMovementResults));
+    }
+  }
+
+  /**
+   * Starts the monitor thread.
+   */
+  void start() {
+    timerThread = new Daemon(new BlocksStorageMovementAttemptResultMonitor());
+    timerThread.setName("BlocksStorageMovementAttemptResultMonitor");
+    timerThread.start();
+  }
+
+  /**
+   * Stops the monitor thread.
+   */
+  public void stop() {
+    spsRunning = false;
+  }
+
+  /**
+   * A monitor class for checking block storage movement result and long waiting
+   * items periodically.
+   */
+  private class BlocksStorageMovementAttemptResultMonitor implements Runnable {
+    @Override
+    public void run() {
+      while (spsRunning) {
+        try {
+          blockStorageMovementResultCheck();
+          blocksStorageMovementUnReportedItemsCheck();
+          Thread.sleep(checkTimeout);
+        } catch (InterruptedException ie) {
+          LOG.debug("BlocksStorageMovementAttemptResultMonitor thread "
+              + "is interrupted.", ie);
+        }
+      }
+    }
+
+    private void blocksStorageMovementUnReportedItemsCheck() {
+      synchronized (storageMovementAttemptedItems) {
+        Iterator<Entry<Long, Long>> iter =
+            storageMovementAttemptedItems.entrySet().iterator();
+        long now = monotonicNow();
+        while (iter.hasNext()) {
+          Entry<Long, Long> entry = iter.next();
+          if (now > entry.getValue() + selfRetryTimeout) {
+            Long blockCollectionID = entry.getKey();
+            synchronized (storageMovementAttemptedResults) {
+              boolean exist = isExistInResult(blockCollectionID);
+              if (!exist) {
+                blockStorageMovementNeeded.add(blockCollectionID);
+              } else {
+                LOG.info("Blocks storage movement results for the"
+                    + " tracking id : " + blockCollectionID
+                    + " is reported from one of the co-ordinating datanode."
+                    + " So, the result will be processed soon.");
+              }
+              iter.remove();
+            }
+          }
+        }
+
+      }
+    }
+
+    private boolean isExistInResult(Long blockCollectionID) {
+      Iterator<BlocksStorageMovementResult> iter =
+          storageMovementAttemptedResults.iterator();
+      while (iter.hasNext()) {
+        BlocksStorageMovementResult storageMovementAttemptedResult =
+            iter.next();
+        if (storageMovementAttemptedResult.getTrackId() == blockCollectionID) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    private void blockStorageMovementResultCheck() {
+      synchronized (storageMovementAttemptedResults) {
+        Iterator<BlocksStorageMovementResult> iter =
+            storageMovementAttemptedResults.iterator();
+        while (iter.hasNext()) {
+          BlocksStorageMovementResult storageMovementAttemptedResult =
+              iter.next();
+          if (storageMovementAttemptedResult
+              .getStatus() == BlocksStorageMovementResult.Status.FAILURE) {
+            blockStorageMovementNeeded
+                .add(storageMovementAttemptedResult.getTrackId());
+            LOG.warn("Blocks storage movement results for the tracking id : "
+                + storageMovementAttemptedResult.getTrackId()
+                + " is reported from co-ordinating datanode, but result"
+                + " status is FAILURE. So, added for retry");
+          } else {
+            synchronized (storageMovementAttemptedItems) {
+              storageMovementAttemptedItems
+                  .remove(storageMovementAttemptedResult.getTrackId());
+            }
+            LOG.info("Blocks storage movement results for the tracking id : "
+                + storageMovementAttemptedResult.getTrackId()
+                + " is reported from co-ordinating datanode. "
+                + "The result status is SUCCESS.");
+          }
+          iter.remove(); // remove from results as processed above
+        }
+      }
+
+    }
+  }
+
+  @VisibleForTesting
+  public int resultsCount() {
+    return storageMovementAttemptedResults.size();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ddfbe11/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 fbe686a..6fa9302 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
@@ -69,6 +69,7 @@ public class StoragePolicySatisfier implements Runnable {
   private final Namesystem namesystem;
   private final BlockManager blockManager;
   private final BlockStorageMovementNeeded storageMovementNeeded;
+  private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
 
   public StoragePolicySatisfier(final Namesystem namesystem,
       final BlockStorageMovementNeeded storageMovementNeeded,
@@ -76,15 +77,22 @@ public class StoragePolicySatisfier implements Runnable {
     this.namesystem = namesystem;
     this.storageMovementNeeded = storageMovementNeeded;
     this.blockManager = blkManager;
+    // TODO: below selfRetryTimeout and checkTimeout can be configurable later
+    // Now, the default values of selfRetryTimeout and checkTimeout are 30mins
+    // and 5mins respectively
+    this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(
+        5 * 60 * 1000, 30 * 60 * 1000, storageMovementNeeded);
   }
 
   /**
-   * Start storage policy satisfier demon thread.
+   * Start storage policy satisfier demon thread. Also start block storage
+   * movements monitor for retry the attempts if needed.
    */
   public void start() {
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
+    this.storageMovementsMonitor.start();
   }
 
   /**
@@ -99,6 +107,7 @@ public class StoragePolicySatisfier implements Runnable {
       storagePolicySatisfierThread.join(3000);
     } catch (InterruptedException ie) {
     }
+    this.storageMovementsMonitor.stop();
   }
 
   @Override
@@ -108,6 +117,7 @@ public class StoragePolicySatisfier implements Runnable {
         Long blockCollectionID = storageMovementNeeded.get();
         if (blockCollectionID != null) {
           computeAndAssignStorageMismatchedBlocksToDNs(blockCollectionID);
+          this.storageMovementsMonitor.add(blockCollectionID);
         }
         // TODO: We can think to make this as configurable later, how frequently
         // we want to check block movements.
@@ -398,11 +408,6 @@ public class StoragePolicySatisfier implements Runnable {
     }
   }
 
-  // TODO: Temporarily keeping the results for assertion. This has to be
-  // revisited as part of HDFS-11029.
-  @VisibleForTesting
-  List<BlocksStorageMovementResult> results = new ArrayList<>();
-
   /**
    * Receives the movement results of collection of blocks associated to a
    * trackId.
@@ -415,6 +420,11 @@ public class StoragePolicySatisfier implements Runnable {
     if (blksMovementResults.length <= 0) {
       return;
     }
-    results.addAll(Arrays.asList(blksMovementResults));
+    storageMovementsMonitor.addResults(blksMovementResults);
+  }
+
+  @VisibleForTesting
+  BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
+    return storageMovementsMonitor;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ddfbe11/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
new file mode 100644
index 0000000..8c70d99
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockStorageMovementAttemptedItems.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.apache.hadoop.util.Time.monotonicNow;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests that block storage movement attempt failures are reported from DN and
+ * processed them correctly or not.
+ */
+public class TestBlockStorageMovementAttemptedItems {
+
+  private BlockStorageMovementAttemptedItems bsmAttemptedItems = null;
+  private BlockStorageMovementNeeded unsatisfiedStorageMovementFiles = null;
+
+  @Before
+  public void setup() {
+    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded();
+    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100, 500,
+        unsatisfiedStorageMovementFiles);
+    bsmAttemptedItems.start();
+  }
+
+  @After
+  public void teardown() {
+    if (bsmAttemptedItems != null) {
+      bsmAttemptedItems.stop();
+    }
+  }
+
+  private boolean checkItemMovedForRetry(Long item, long retryTimeout)
+      throws InterruptedException {
+    long stopTime = monotonicNow() + (retryTimeout * 2);
+    boolean isItemFound = false;
+    while (monotonicNow() < (stopTime)) {
+      Long ele = null;
+      while ((ele = unsatisfiedStorageMovementFiles.get()) != null) {
+        if (item.longValue() == ele.longValue()) {
+          isItemFound = true;
+          break;
+        }
+      }
+      if (!isItemFound) {
+        Thread.sleep(100);
+      } else {
+        break;
+      }
+    }
+    return isItemFound;
+  }
+
+  @Test(timeout = 30000)
+  public void testAddResultWithFailureResult() throws Exception {
+    Long item = new Long(1234);
+    bsmAttemptedItems.add(item);
+    bsmAttemptedItems.addResults(
+        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
+            item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
+    assertTrue(checkItemMovedForRetry(item, 200));
+  }
+
+  @Test(timeout = 30000)
+  public void testAddResultWithSucessResult() throws Exception {
+    Long item = new Long(1234);
+    bsmAttemptedItems.add(item);
+    bsmAttemptedItems.addResults(
+        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
+            item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
+    assertFalse(checkItemMovedForRetry(item, 200));
+  }
+
+  @Test(timeout = 30000)
+  public void testNoResultAdded() throws Exception {
+    Long item = new Long(1234);
+    bsmAttemptedItems.add(item);
+    // After selfretry timeout, it should be added back for retry
+    assertTrue(checkItemMovedForRetry(item, 600));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ddfbe11/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 cbfdfc6..6f5c717 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
@@ -174,8 +174,6 @@ public class TestStoragePolicySatisfier {
       waitExpectedStorageType(file, StorageType.SSD, distributedFS, 1, 30000);
       waitExpectedStorageType(file, StorageType.DISK, distributedFS, 2, 30000);
 
-      // TODO: Temporarily using the results from StoragePolicySatisfier class.
-      // This has to be revisited as part of HDFS-11029.
       waitForBlocksMovementResult(1, 30000);
     } finally {
       hdfsCluster.shutdown();
@@ -190,8 +188,10 @@ public class TestStoragePolicySatisfier {
       @Override
       public Boolean get() {
         LOG.info("expectedResultsCount={} actualResultsCount={}",
-            expectedResultsCount, sps.results.size());
-        return expectedResultsCount == sps.results.size();
+            expectedResultsCount,
+            sps.getAttemptedItemsMonitor().resultsCount());
+        return expectedResultsCount == sps.getAttemptedItemsMonitor()
+            .resultsCount();
       }
     }, 100, timeout);
   }


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


[33/50] [abbrv] hadoop git commit: HDFS-11670: [SPS]: Add CLI command for satisfy storage policy operations. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-11670: [SPS]: Add CLI command for satisfy storage policy operations. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 71913478de97089e2776812a6692f4db4308f20b
Parents: 5262c65
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jun 19 17:16:49 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:13 2017 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   | 93 +++++++++++++++++++-
 .../src/site/markdown/ArchivalStorage.md        | 21 +++++
 .../src/site/markdown/HDFSCommands.md           |  2 +
 .../hdfs/tools/TestStoragePolicyCommands.java   | 43 ++++++++-
 4 files changed, 157 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/71913478/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 aeb10d9..662957c 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
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -32,6 +33,8 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
 import java.io.FileNotFoundException;
+import com.google.common.base.Joiner;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
@@ -245,6 +248,92 @@ public class StoragePolicyAdmin extends Configured implements Tool {
     }
   }
 
+  /** Command to schedule blocks to move based on specified policy. */
+  private static class SatisfyStoragePolicyCommand implements
+      AdminHelper.Command {
+    @Override
+    public String getName() {
+      return "-satisfyStoragePolicy";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " -path <path>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      TableListing listing = AdminHelper.getOptionDescriptionListing();
+      listing.addRow("<path>", "The path of the file/directory to satisfy"
+          + " storage policy");
+      return getShortUsage() + "\n" +
+          "Schedule blocks to move based on file/directory policy.\n\n" +
+          listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      final String path = StringUtils.popOptionWithArgument("-path", args);
+      if (path == null) {
+        System.err.println("Please specify the path for setting the storage " +
+            "policy.\nUsage: " + getLongUsage());
+        return 1;
+      }
+
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      try {
+        dfs.satisfyStoragePolicy(new Path(path));
+        System.out.println("Scheduled blocks to move based on the current"
+            + " storage policy on " + path);
+      } catch (Exception e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
+  /** Command to check storage policy satisfier status. */
+  private static class IsSPSRunningCommand implements AdminHelper.Command {
+    @Override
+    public String getName() {
+      return "-isSPSRunning";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + "]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      return getShortUsage() + "\n" +
+          "Check the status of Storage Policy Statisfier.\n\n";
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      if (!args.isEmpty()) {
+        System.err.print("Can't understand arguments: "
+            + Joiner.on(" ").join(args) + "\n");
+        System.err.println("Usage is " + getLongUsage());
+        return 1;
+      }
+      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      try {
+        if(dfs.getClient().isStoragePolicySatisfierRunning()){
+          System.out.println("yes");
+        }else{
+          System.out.println("no");
+        }
+      } catch (Exception e) {
+        System.err.println(AdminHelper.prettifyException(e));
+        return 2;
+      }
+      return 0;
+    }
+  }
+
   /* Command to unset the storage policy set for a file/directory */
   private static class UnsetStoragePolicyCommand
       implements AdminHelper.Command {
@@ -295,6 +384,8 @@ public class StoragePolicyAdmin extends Configured implements Tool {
       new ListStoragePoliciesCommand(),
       new SetStoragePolicyCommand(),
       new GetStoragePolicyCommand(),
-      new UnsetStoragePolicyCommand()
+      new UnsetStoragePolicyCommand(),
+      new SatisfyStoragePolicyCommand(),
+      new IsSPSRunningCommand()
   };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71913478/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 91ad107..f1895fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -169,5 +169,26 @@ Get the storage policy of a file or a directory.
 |:---- |:---- |
 | `-path <path>` | The path referring to either a directory or a file. |
 
+### Satisfy Storage Policy
+
+Schedule blocks to move based on file/directory policy. This command applicable only to the given path and its immediate children. Sub-directories won't be considered for satisfying the policy.
+
+* Command:
+
+        hdfs storagepolicies -satisfyStoragePolicy -path <path>
+
+* Arguments:
+
+| | |
+|:---- |:---- |
+| `-path <path>` | The path referring to either a directory or a file. |
+
+### SPS Running Status
+
+Check the running status of Storage Policy Satisfier in namenode. If it is running, return 'yes'. Otherwise return 'no'.
+
+* Command:
+
+        hdfs storagepolicies -isSPSRunning
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71913478/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 d8462c1..c5dd740 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -601,6 +601,8 @@ Usage:
           [-setStoragePolicy -path <path> -policy <policy>]
           [-getStoragePolicy -path <path>]
           [-unsetStoragePolicy -path <path>]
+          [-satisfyStoragePolicy -path <path>]
+          [-isSPSRunning]
           [-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/71913478/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 f31c739..59f9083 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
@@ -23,6 +23,8 @@ import java.net.URISyntaxException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -46,7 +48,10 @@ public class TestStoragePolicyCommands {
   @Before
   public void clusterSetUp() throws IOException, URISyntaxException {
     conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).build();
+    StorageType[][] newtypes = new StorageType[][] {
+        {StorageType.ARCHIVE, StorageType.DISK}};
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL)
+        .storageTypes(newtypes).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
   }
@@ -158,4 +163,40 @@ public class TestStoragePolicyCommands {
     DFSTestUtil.toolRun(admin, "-getStoragePolicy -path /fooz", 2,
         "File/Directory does not exist: /fooz");
   }
+
+  @Test
+  public void testStoragePolicySatisfierCommand() throws Exception {
+    final String file = "/testStoragePolicySatisfierCommand";
+    DFSTestUtil.createFile(fs, new Path(file), SIZE, REPL, 0);
+
+    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
+    DFSTestUtil.toolRun(admin, "-getStoragePolicy -path " + file, 0,
+        "The storage policy of " + file + " is unspecified");
+
+    DFSTestUtil.toolRun(admin,
+        "-setStoragePolicy -path " + file + " -policy COLD", 0,
+        "Set storage policy COLD on " + file.toString());
+
+    DFSTestUtil.toolRun(admin, "-satisfyStoragePolicy -path " + file, 0,
+        "Scheduled blocks to move based on the current storage policy on "
+            + file.toString());
+
+    DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
+        fs);
+  }
+
+  @Test
+  public void testIsSPSRunningCommand() throws Exception {
+    final String file = "/testIsSPSRunningCommand";
+    DFSTestUtil.createFile(fs, new Path(file), SIZE, REPL, 0);
+    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
+    DFSTestUtil.toolRun(admin, "-isSPSRunning", 0, "yes");
+    cluster.getNameNode().reconfigureProperty(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "false");
+    cluster.waitActive();
+    DFSTestUtil.toolRun(admin, "-isSPSRunning", 0, "no");
+    // Test with unnecessary args
+    DFSTestUtil.toolRun(admin, "-isSPSRunning 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


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

Posted by ra...@apache.org.
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/9395fcf4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9395fcf4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9395fcf4

Branch: refs/heads/HDFS-10285
Commit: 9395fcf4d1d53634d132a28c0932a90bcfabceef
Parents: ad422f1
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Thu Aug 17 13:21:07 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:56 2017 +0530

----------------------------------------------------------------------
 .../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/9395fcf4/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 38be348b..bc6e7a4 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -62,7 +62,7 @@ function hadoop_usage
   hadoop_add_subcommand "portmap" daemon "run a portmap service"
   hadoop_add_subcommand "secondarynamenode" daemon "run the DFS secondary namenode"
   hadoop_add_subcommand "snapshotDiff" client "diff two snapshots of a directory or diff the current directory contents with a snapshot"
-  hadoop_add_subcommand "storagepolicies" admin "list/get/set block storage policies"
+  hadoop_add_subcommand "storagepolicies" admin "list/get/set/satisfyStoragePolicy block storage policies"
   hadoop_add_subcommand "version" client "print the version"
   hadoop_add_subcommand "zkfc" daemon "run the ZK Failover Controller daemon"
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" false

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9395fcf4/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 fcac7a5..5c5a3d9 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
@@ -570,10 +570,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/9395fcf4/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 8349465..68fbf6a 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
@@ -434,6 +434,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();
 
@@ -442,15 +444,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);
@@ -478,24 +474,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(
@@ -721,15 +708,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();
@@ -4980,46 +4962,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/9395fcf4/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/9395fcf4/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 c5ab56b..33d7b92 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
@@ -1285,7 +1285,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             edekCacheLoaderDelay, edekCacheLoaderInterval);
       }
 
-      blockManager.activateSPS();
+      blockManager.startSPS();
     } finally {
       startingActiveService = false;
       blockManager.checkSafeMode();
@@ -1316,7 +1316,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       if (blockManager != null) {
-        blockManager.deactivateSPS();
+        blockManager.stopSPS(true);
       }
       stopSecretManager();
       leaseManager.stopMonitor();
@@ -2170,6 +2170,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 {
@@ -2187,16 +2189,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/9395fcf4/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 63eaa3a..8542a1a 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
@@ -158,7 +158,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;
@@ -291,7 +291,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["
@@ -2052,8 +2052,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));
@@ -2137,29 +2137,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/9395fcf4/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/9395fcf4/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/9395fcf4/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 662957c..c351410 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
@@ -249,8 +249,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";
@@ -294,10 +294,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
@@ -386,6 +387,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/9395fcf4/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 8beadd2..22d4bd8 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
@@ -4344,13 +4344,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/9395fcf4/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/9395fcf4/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 c5dd740..9d687f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -602,7 +602,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/9395fcf4/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/9395fcf4/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 17fd37c..ca1b5eb 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/9395fcf4/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 d548649..f29aa09 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/9395fcf4/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/9395fcf4/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 3e7f2e3..b4a461a 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.hdfs.DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE_DEFAULT;
 
@@ -223,63 +224,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");
@@ -288,32 +325,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));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9395fcf4/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/9395fcf4/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/9395fcf4/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/9395fcf4/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/9395fcf4/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 59f9083..1a38105 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
@@ -48,6 +48,8 @@ public class TestStoragePolicyCommands {
   @Before
   public void clusterSetUp() throws IOException, URISyntaxException {
     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)
@@ -164,7 +166,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);
@@ -185,18 +187,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


[16/50] [abbrv] hadoop git commit: HDFS-11193 : [SPS]: Erasure coded files should be considered for satisfying storage policy. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-11193 : [SPS]: Erasure coded files should be considered for satisfying storage policy. 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/470e7f73
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/470e7f73
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/470e7f73

Branch: refs/heads/HDFS-10285
Commit: 470e7f73dc06c5cc0604b2b0f13d725a02af3b1e
Parents: fe2cb59
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Thu Jan 5 09:30:39 2017 -0800
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:35 2017 +0530

----------------------------------------------------------------------
 .../blockmanagement/BlockInfoStriped.java       |  10 +
 .../server/namenode/StoragePolicySatisfier.java |  76 ++-
 ...stStoragePolicySatisfierWithStripedFile.java | 469 +++++++++++++++++++
 3 files changed, 551 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/470e7f73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 790cd77..8bc63c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -244,6 +244,10 @@ public class BlockInfoStriped extends BlockInfo {
     return true;
   }
 
+  /**
+   * This class contains datanode storage information and block index in the
+   * block group.
+   */
   public static class StorageAndBlockIndex {
     private final DatanodeStorageInfo storage;
     private final byte blockIndex;
@@ -253,10 +257,16 @@ public class BlockInfoStriped extends BlockInfo {
       this.blockIndex = blockIndex;
     }
 
+    /**
+     * @return storage in the datanode.
+     */
     public DatanodeStorageInfo getStorage() {
       return storage;
     }
 
+    /**
+     * @return block index in the block group.
+     */
     public byte getBlockIndex() {
       return blockIndex;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/470e7f73/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 26e0775..a854bd7 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
@@ -36,6 +36,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.balancer.Matcher;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped.StorageAndBlockIndex;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -43,6 +45,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -226,8 +229,26 @@ public class StoragePolicySatisfier implements Runnable {
 
     for (int i = 0; i < blocks.length; i++) {
       BlockInfo blockInfo = blocks[i];
-      List<StorageType> expectedStorageTypes = existingStoragePolicy
+      List<StorageType> expectedStorageTypes;
+      if (blockInfo.isStriped()) {
+        if (ErasureCodingPolicyManager
+            .checkStoragePolicySuitableForECStripedMode(
+                existingStoragePolicyID)) {
+          expectedStorageTypes = existingStoragePolicy
+              .chooseStorageTypes((short) blockInfo.getCapacity());
+        } else {
+          // Currently we support only limited policies (HOT, COLD, ALLSSD)
+          // for EC striped mode files. SPS will ignore to move the blocks if
+          // the storage policy is not in EC Striped mode supported policies
+          LOG.warn("The storage policy " + existingStoragePolicy.getName()
+              + " is not suitable for Striped EC files. "
+              + "So, ignoring to move the blocks");
+          return false;
+        }
+      } else {
+        expectedStorageTypes = existingStoragePolicy
             .chooseStorageTypes(blockInfo.getReplication());
+      }
       foundMatchingTargetNodesForAllBlocks |= computeBlockMovingInfos(
           blockMovingInfos, blockInfo, expectedStorageTypes);
     }
@@ -439,12 +460,18 @@ public class StoragePolicySatisfier implements Runnable {
     if (sourceNodes.size() <= 0) {
       return blkMovingInfos;
     }
-    buildBlockMovingInfos(blockInfo, sourceNodes, sourceStorageTypes,
-        targetNodes, targetStorageTypes, blkMovingInfos);
+
+    if (blockInfo.isStriped()) {
+      buildStripedBlockMovingInfos(blockInfo, sourceNodes, sourceStorageTypes,
+          targetNodes, targetStorageTypes, blkMovingInfos);
+    } else {
+      buildContinuousBlockMovingInfos(blockInfo, sourceNodes,
+          sourceStorageTypes, targetNodes, targetStorageTypes, blkMovingInfos);
+    }
     return blkMovingInfos;
   }
 
-  private void buildBlockMovingInfos(BlockInfo blockInfo,
+  private void buildContinuousBlockMovingInfos(BlockInfo blockInfo,
       List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
       List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes,
       List<BlockMovingInfo> blkMovingInfos) {
@@ -458,6 +485,47 @@ public class StoragePolicySatisfier implements Runnable {
     blkMovingInfos.add(blkMovingInfo);
   }
 
+  private void buildStripedBlockMovingInfos(BlockInfo blockInfo,
+      List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
+      List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes,
+      List<BlockMovingInfo> blkMovingInfos) {
+    // For a striped block, it needs to construct internal block at the given
+    // index of a block group. Here it is iterating over all the block indices
+    // and construct internal blocks which can be then considered for block
+    // movement.
+    BlockInfoStriped sBlockInfo = (BlockInfoStriped) blockInfo;
+    for (StorageAndBlockIndex si : sBlockInfo.getStorageAndIndexInfos()) {
+      if (si.getBlockIndex() >= 0) {
+        DatanodeDescriptor dn = si.getStorage().getDatanodeDescriptor();
+        DatanodeInfo[] srcNode = new DatanodeInfo[1];
+        StorageType[] srcStorageType = new StorageType[1];
+        DatanodeInfo[] targetNode = new DatanodeInfo[1];
+        StorageType[] targetStorageType = new StorageType[1];
+        for (int i = 0; i < sourceNodes.size(); i++) {
+          DatanodeInfo node = sourceNodes.get(i);
+          if (node.equals(dn)) {
+            srcNode[0] = node;
+            srcStorageType[0] = sourceStorageTypes.get(i);
+            targetNode[0] = targetNodes.get(i);
+            targetStorageType[0] = targetStorageTypes.get(i);
+
+            // construct internal block
+            long blockId = blockInfo.getBlockId() + si.getBlockIndex();
+            long numBytes = StripedBlockUtil.getInternalBlockLength(
+                sBlockInfo.getNumBytes(), sBlockInfo.getCellSize(),
+                sBlockInfo.getDataBlockNum(), si.getBlockIndex());
+            Block blk = new Block(blockId, numBytes,
+                blockInfo.getGenerationStamp());
+            BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, srcNode,
+                targetNode, srcStorageType, targetStorageType);
+            blkMovingInfos.add(blkMovingInfo);
+            break; // found matching source-target nodes
+          }
+        }
+      }
+    }
+  }
+
   /**
    * Choose the target storage within same datanode if possible.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/470e7f73/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
new file mode 100644
index 0000000..5f8639f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithStripedFile.java
@@ -0,0 +1,469 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Supplier;
+
+/**
+ * Tests that StoragePolicySatisfier daemon is able to check the striped blocks
+ * to be moved and finding its expected target locations in order to satisfy the
+ * storage policy.
+ */
+public class TestStoragePolicySatisfierWithStripedFile {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestStoragePolicySatisfierWithStripedFile.class);
+
+  private final int stripesPerBlock = 2;
+
+  private ErasureCodingPolicy ecPolicy;
+  private int dataBlocks;
+  private int parityBlocks;
+  private int cellSize;
+  private int defaultStripeBlockSize;
+
+  private ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  }
+
+  /**
+   * Initialize erasure coding policy.
+   */
+  @Before
+  public void init(){
+    ecPolicy = getEcPolicy();
+    dataBlocks = ecPolicy.getNumDataUnits();
+    parityBlocks = ecPolicy.getNumParityUnits();
+    cellSize = ecPolicy.getCellSize();
+    defaultStripeBlockSize = cellSize * stripesPerBlock;
+  }
+
+  /**
+   * Tests to verify that all the striped blocks(data + parity blocks) are
+   * moving to satisfy the storage policy.
+   */
+  @Test(timeout = 300000)
+  public void testMoverWithFullStripe() throws Exception {
+    // start 10 datanodes
+    int numOfDatanodes = 10;
+    int storagesPerDatanode = 2;
+    long capacity = 20 * defaultStripeBlockSize;
+    long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
+    for (int i = 0; i < numOfDatanodes; i++) {
+      for (int j = 0; j < storagesPerDatanode; j++) {
+        capacities[i][j] = capacity;
+      }
+    }
+
+    final Configuration conf = new HdfsConfiguration();
+    initConfWithStripe(conf, defaultStripeBlockSize);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numOfDatanodes)
+        .storagesPerDatanode(storagesPerDatanode)
+        .storageTypes(new StorageType[][]{
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE}})
+        .storageCapacities(capacities)
+        .build();
+
+    HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+    try {
+      cluster.waitActive();
+
+      // set "/bar" directory with HOT storage policy.
+      ClientProtocol client = NameNodeProxies.createProxy(conf,
+          cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
+      String barDir = "/bar";
+      client.mkdirs(barDir, new FsPermission((short) 777), true);
+      client.setStoragePolicy(barDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
+      // set an EC policy on "/bar" directory
+      client.setErasureCodingPolicy(barDir, null);
+
+      // write file to barDir
+      final String fooFile = "/bar/foo";
+      long fileLen = cellSize * dataBlocks;
+      DFSTestUtil.createFile(cluster.getFileSystem(), new Path(fooFile),
+          fileLen, (short) 3, 0);
+
+      // verify storage types and locations
+      LocatedBlocks locatedBlocks = client.getBlockLocations(fooFile, 0,
+          fileLen);
+      for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
+        for (StorageType type : lb.getStorageTypes()) {
+          Assert.assertEquals(StorageType.DISK, type);
+        }
+      }
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
+          dataBlocks + parityBlocks);
+
+      // start 5 more datanodes
+      int numOfNewDatanodes = 5;
+      capacities = new long[numOfNewDatanodes][storagesPerDatanode];
+      for (int i = 0; i < numOfNewDatanodes; i++) {
+        for (int j = 0; j < storagesPerDatanode; j++) {
+          capacities[i][j] = capacity;
+        }
+      }
+      cluster.startDataNodes(conf, 5,
+          new StorageType[][]{
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}},
+          true, null, null, null, capacities, null, false, false, false, null);
+      cluster.triggerHeartbeats();
+
+      // move file to ARCHIVE
+      client.setStoragePolicy(barDir, "COLD");
+      hdfsAdmin.satisfyStoragePolicy(new Path(fooFile));
+      LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
+      cluster.triggerHeartbeats();
+
+      waitForBlocksMovementResult(cluster, 1, 60000);
+      // verify storage types and locations
+      waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 9,
+          9, 60000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests to verify that only few datanodes are available and few striped
+   * blocks are able to move. Others are still trying to find available nodes.
+   *
+   * For example, we have 3 nodes A(disk, disk), B(disk, disk), C(disk, archive)
+   *
+   * Assume a block with storage locations A(disk), B(disk), C(disk). Now, set
+   * policy as COLD and invoked {@link HdfsAdmin#satisfyStoragePolicy(Path)},
+   * while choosing the target node for A, it shouldn't choose C. For C, it
+   * should do local block movement as it has ARCHIVE storage type.
+   */
+  @Test(timeout = 300000)
+  public void testWhenOnlyFewTargetNodesAreAvailableToSatisfyStoragePolicy()
+      throws Exception {
+    // start 10 datanodes
+    int numOfDatanodes = 10;
+    int storagesPerDatanode = 2;
+    long capacity = 20 * defaultStripeBlockSize;
+    long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
+    for (int i = 0; i < numOfDatanodes; i++) {
+      for (int j = 0; j < storagesPerDatanode; j++) {
+        capacities[i][j] = capacity;
+      }
+    }
+
+    final Configuration conf = new HdfsConfiguration();
+    initConfWithStripe(conf, defaultStripeBlockSize);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numOfDatanodes)
+        .storagesPerDatanode(storagesPerDatanode)
+        .storageTypes(new StorageType[][]{
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE}})
+        .storageCapacities(capacities)
+        .build();
+
+    HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+    try {
+      cluster.waitActive();
+
+      // set "/bar" directory with HOT storage policy.
+      ClientProtocol client = NameNodeProxies.createProxy(conf,
+          cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
+      String barDir = "/bar";
+      client.mkdirs(barDir, new FsPermission((short) 777), true);
+      client.setStoragePolicy(barDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
+      // set an EC policy on "/bar" directory
+      client.setErasureCodingPolicy(barDir, null);
+
+      // write file to barDir
+      final String fooFile = "/bar/foo";
+      long fileLen = cellSize * dataBlocks;
+      DFSTestUtil.createFile(cluster.getFileSystem(), new Path(fooFile),
+          fileLen, (short) 3, 0);
+
+      // verify storage types and locations
+      LocatedBlocks locatedBlocks = client.getBlockLocations(fooFile, 0,
+          fileLen);
+      for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
+        for (StorageType type : lb.getStorageTypes()) {
+          Assert.assertEquals(StorageType.DISK, type);
+        }
+      }
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
+          dataBlocks + parityBlocks);
+
+      // start 2 more datanodes
+      int numOfNewDatanodes = 2;
+      capacities = new long[numOfNewDatanodes][storagesPerDatanode];
+      for (int i = 0; i < numOfNewDatanodes; i++) {
+        for (int j = 0; j < storagesPerDatanode; j++) {
+          capacities[i][j] = capacity;
+        }
+      }
+      cluster.startDataNodes(conf, 2,
+          new StorageType[][]{
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}},
+          true, null, null, null, capacities, null, false, false, false, null);
+      cluster.triggerHeartbeats();
+
+      // Move file to ARCHIVE. Only 5 datanodes are available with ARCHIVE
+      // storage type.
+      client.setStoragePolicy(barDir, "COLD");
+      hdfsAdmin.satisfyStoragePolicy(new Path(fooFile));
+      LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
+      cluster.triggerHeartbeats();
+
+      waitForBlocksMovementResult(cluster, 1, 60000);
+      waitForAttemptedItems(cluster, 1, 30000);
+      // verify storage types and locations.
+      waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 5,
+          9, 60000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests to verify that for the given path, no blocks under the given path
+   * will be scheduled for block movement as there are no available datanode
+   * with required storage type.
+   *
+   * For example, there are two block for a file:
+   *
+   * File1 => blk_1[locations=A(DISK),B(DISK),C(DISK)],
+   * blk_2[locations=A(DISK),B(DISK),C(DISK)]. Now, set storage policy to COLD.
+   * No datanode is available with storage type ARCHIVE.
+   *
+   * SPS won't schedule any block movement for this path.
+   */
+  @Test(timeout = 300000)
+  public void testWhenNoTargetDatanodeToSatisfyStoragePolicy()
+      throws Exception {
+    // start 10 datanodes
+    int numOfDatanodes = 10;
+    int storagesPerDatanode = 2;
+    long capacity = 20 * defaultStripeBlockSize;
+    long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
+    for (int i = 0; i < numOfDatanodes; i++) {
+      for (int j = 0; j < storagesPerDatanode; j++) {
+        capacities[i][j] = capacity;
+      }
+    }
+
+    final Configuration conf = new HdfsConfiguration();
+    initConfWithStripe(conf, defaultStripeBlockSize);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numOfDatanodes)
+        .storagesPerDatanode(storagesPerDatanode)
+        .storageTypes(new StorageType[][]{
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK}})
+        .storageCapacities(capacities)
+        .build();
+
+    HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+    try {
+      cluster.waitActive();
+
+      // set "/bar" directory with HOT storage policy.
+      ClientProtocol client = NameNodeProxies.createProxy(conf,
+          cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
+      String barDir = "/bar";
+      client.mkdirs(barDir, new FsPermission((short) 777), true);
+      client.setStoragePolicy(barDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
+      // set an EC policy on "/bar" directory
+      client.setErasureCodingPolicy(barDir, null);
+
+      // write file to barDir
+      final String fooFile = "/bar/foo";
+      long fileLen = cellSize * dataBlocks;
+      DFSTestUtil.createFile(cluster.getFileSystem(), new Path(fooFile),
+          fileLen, (short) 3, 0);
+
+      // verify storage types and locations
+      LocatedBlocks locatedBlocks = client.getBlockLocations(fooFile, 0,
+          fileLen);
+      for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
+        for (StorageType type : lb.getStorageTypes()) {
+          Assert.assertEquals(StorageType.DISK, type);
+        }
+      }
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
+          dataBlocks + parityBlocks);
+
+      // Move file to ARCHIVE. Only 5 datanodes are available with ARCHIVE
+      // storage type.
+      client.setStoragePolicy(barDir, "COLD");
+      hdfsAdmin.satisfyStoragePolicy(new Path(fooFile));
+      LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
+      cluster.triggerHeartbeats();
+
+      waitForAttemptedItems(cluster, 1, 30000);
+      // verify storage types and locations.
+      waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.DISK, 9, 9,
+          60000);
+      waitForAttemptedItems(cluster, 1, 30000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void waitForAttemptedItems(MiniDFSCluster cluster,
+      long expectedBlkMovAttemptedCount, int timeout)
+          throws TimeoutException, InterruptedException {
+    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("expectedAttemptedItemsCount={} actualAttemptedItemsCount={}",
+            expectedBlkMovAttemptedCount,
+            sps.getAttemptedItemsMonitor().getAttemptedItemsCount());
+        return sps.getAttemptedItemsMonitor()
+            .getAttemptedItemsCount() == expectedBlkMovAttemptedCount;
+      }
+    }, 100, timeout);
+  }
+
+  private static void initConfWithStripe(Configuration conf,
+      int stripeBlockSize) {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, stripeBlockSize);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
+        1L);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
+        false);
+  }
+
+  // Check whether the Block movement has been successfully completed to satisfy
+  // the storage policy for the given file.
+  private void waitExpectedStorageType(MiniDFSCluster cluster,
+      final String fileName, long fileLen,
+      final StorageType expectedStorageType, int expectedStorageCount,
+      int expectedBlkLocationCount, int timeout) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        int actualStorageCount = 0;
+        try {
+          LocatedBlocks locatedBlocks = cluster.getFileSystem().getClient()
+              .getLocatedBlocks(fileName, 0, fileLen);
+          for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
+            LOG.info("LocatedBlocks => Size {}, locs {}",
+                lb.getLocations().length, lb);
+            if (lb.getLocations().length > expectedBlkLocationCount) {
+              return false;
+            }
+            for (StorageType storageType : lb.getStorageTypes()) {
+              if (expectedStorageType == storageType) {
+                actualStorageCount++;
+              } else {
+                LOG.info("Expected storage type {} and actual {}",
+                    expectedStorageType, storageType);
+              }
+            }
+          }
+          LOG.info(
+              expectedStorageType + " replica count, expected={} and actual={}",
+              expectedStorageCount, actualStorageCount);
+        } catch (IOException e) {
+          LOG.error("Exception while getting located blocks", e);
+          return false;
+        }
+        return expectedStorageCount == actualStorageCount;
+      }
+    }, 100, timeout);
+  }
+
+  // Check whether the block movement result has been arrived at the
+  // Namenode(SPS).
+  private void waitForBlocksMovementResult(MiniDFSCluster cluster,
+      long expectedBlkMovResultsCount, int timeout)
+          throws TimeoutException, InterruptedException {
+    BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    Assert.assertNotNull("Failed to get SPS object reference!", sps);
+
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("expectedResultsCount={} actualResultsCount={}",
+            expectedBlkMovResultsCount,
+            sps.getAttemptedItemsMonitor().resultsCount());
+        return sps.getAttemptedItemsMonitor()
+            .resultsCount() == expectedBlkMovResultsCount;
+      }
+    }, 100, timeout);
+  }
+}


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


[48/50] [abbrv] hadoop git commit: HDFS-12310: [SPS]: Provide an option to track the status of in progress requests. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-12310: [SPS]: Provide an option to track the status of in progress requests. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: ee0e8b0edc01654209d4d9c008fd255bc293da82
Parents: e6e61b7
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Nov 3 08:18:14 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:31:24 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  22 ++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  21 ++++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  27 +++++
 .../ClientNamenodeProtocolTranslatorPB.java     |  20 ++++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  33 ++++++
 .../src/main/proto/ClientNamenodeProtocol.proto |  17 ++-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  23 +++-
 .../server/blockmanagement/BlockManager.java    |  12 ++
 .../namenode/BlockStorageMovementNeeded.java    | 109 +++++++++++++++++++
 .../hdfs/server/namenode/NameNodeRpcServer.java |  13 ++-
 .../server/namenode/StoragePolicySatisfier.java |   8 ++
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |  35 +++++-
 .../src/site/markdown/ArchivalStorage.md        |   3 +-
 .../TestPersistentStoragePolicySatisfier.java   |   2 +-
 .../namenode/TestStoragePolicySatisfier.java    |  67 ++++++++++++
 .../hdfs/tools/TestStoragePolicyCommands.java   |  18 +++
 16 files changed, 424 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 376ff60..77d0b03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -123,6 +123,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
@@ -3053,4 +3054,25 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     checkOpen();
     return new OpenFilesIterator(namenode, tracer);
   }
+
+  /**
+   * Check the storage policy satisfy status of the path for which
+   * {@link DFSClient#satisfyStoragePolicy(String)} is called.
+   *
+   * @return Storage policy satisfy status.
+   *         <ul>
+   *         <li>PENDING if path is in queue and not processed for satisfying
+   *         the policy.</li>
+   *         <li>IN_PROGRESS if satisfying the storage policy for path.</li>
+   *         <li>SUCCESS if storage policy satisfied for the path.</li>
+   *         <li>NOT_AVAILABLE if
+   *         {@link DFSClient#satisfyStoragePolicy(String)} not called for
+   *         path or SPS work is already finished.</li>
+   *         </ul>
+   * @throws IOException
+   */
+  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException {
+    return namenode.checkStoragePolicySatisfyPathStatus(path);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index e72975d..e6c3d4a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
@@ -1721,4 +1722,24 @@ public interface ClientProtocol {
    */
   @Idempotent
   boolean isStoragePolicySatisfierRunning() throws IOException;
+
+  /**
+   * Check the storage policy satisfy status of the path for which
+   * {@link ClientProtocol#satisfyStoragePolicy(String)} is called.
+   *
+   * @return Storage policy satisfy status.
+   *         <ul>
+   *         <li>PENDING if path is in queue and not processed for satisfying
+   *         the policy.</li>
+   *         <li>IN_PROGRESS if satisfying the storage policy for path.</li>
+   *         <li>SUCCESS if storage policy satisfied for the path.</li>
+   *         <li>NOT_AVAILABLE if
+   *         {@link ClientProtocol#satisfyStoragePolicy(String)} not called for
+   *         path or SPS work is already finished.</li>
+   *         </ul>
+   * @throws IOException
+   */
+  @Idempotent
+  StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 8245d1b..aa0496e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -125,6 +125,33 @@ public final class HdfsConstants {
     SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET, SAFEMODE_FORCE_EXIT
   }
 
+  /**
+   * Storage policy satisfy path status.
+   */
+  public enum StoragePolicySatisfyPathStatus {
+    /**
+     * Scheduled but not yet processed. This will come only in case of
+     * directory. Directory will be added first in "pendingWorkForDirectory"
+     * queue and then later it is processed recursively.
+     */
+    PENDING,
+
+    /**
+     * Satisfying the storage policy for path.
+     */
+    IN_PROGRESS,
+
+    /**
+     * Storage policy satisfied for the path.
+     */
+    SUCCESS,
+
+    /**
+     * Status not available.
+     */
+    NOT_AVAILABLE
+  }
+
   public enum RollingUpgradeAction {
     QUERY, PREPARE, FINALIZE;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 269d095..78fe792 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -97,6 +98,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Append
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckStoragePolicySatisfyPathStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckStoragePolicySatisfyPathStatusResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
@@ -234,6 +237,7 @@ import org.apache.hadoop.security.token.Token;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Message;
 import com.google.protobuf.ServiceException;
+
 import org.apache.hadoop.util.concurrent.AsyncGet;
 
 /**
@@ -1917,4 +1921,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException {
+    try {
+      CheckStoragePolicySatisfyPathStatusRequestProto request =
+          CheckStoragePolicySatisfyPathStatusRequestProto.newBuilder()
+          .setSrc(path)
+          .build();
+      CheckStoragePolicySatisfyPathStatusResponseProto response = rpcProxy
+          .checkStoragePolicySatisfyPathStatus(null, request);
+      return PBHelperClient.convert(response.getStatus());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index fbc6dbf..b396c3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -123,6 +123,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheF
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckStoragePolicySatisfyPathStatusResponseProto.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
@@ -3238,4 +3239,36 @@ public class PBHelperClient {
     }
     return ret;
   }
+
+  public static StoragePolicySatisfyPathStatus convert(
+      HdfsConstants.StoragePolicySatisfyPathStatus status) {
+    switch (status) {
+    case PENDING:
+      return StoragePolicySatisfyPathStatus.PENDING;
+    case IN_PROGRESS:
+      return StoragePolicySatisfyPathStatus.IN_PROGRESS;
+    case SUCCESS:
+      return StoragePolicySatisfyPathStatus.SUCCESS;
+    case NOT_AVAILABLE:
+      return StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
+    default:
+      throw new IllegalArgumentException("Unexpected SPSStatus :" + status);
+    }
+  }
+
+  public static HdfsConstants.StoragePolicySatisfyPathStatus convert(
+      StoragePolicySatisfyPathStatus status) {
+    switch (status) {
+    case PENDING:
+      return HdfsConstants.StoragePolicySatisfyPathStatus.PENDING;
+    case IN_PROGRESS:
+      return HdfsConstants.StoragePolicySatisfyPathStatus.IN_PROGRESS;
+    case SUCCESS:
+      return HdfsConstants.StoragePolicySatisfyPathStatus.SUCCESS;
+    case NOT_AVAILABLE:
+      return HdfsConstants.StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
+    default:
+      throw new IllegalArgumentException("Unexpected SPSStatus :" + status);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 68a6124..0e7f779 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -470,7 +470,6 @@ message RollingUpgradeInfoProto {
 message RollingUpgradeResponseProto {
   optional RollingUpgradeInfoProto rollingUpgradeInfo= 1;
 }
-
 message ListCorruptFileBlocksRequestProto {
   required string path = 1;
   optional string cookie = 2;
@@ -818,6 +817,20 @@ message IsStoragePolicySatisfierRunningResponseProto {
   required bool running = 1;
 }
 
+message CheckStoragePolicySatisfyPathStatusRequestProto {  // no parameters
+  required string src = 1;
+}
+
+message CheckStoragePolicySatisfyPathStatusResponseProto {
+  enum StoragePolicySatisfyPathStatus {
+    PENDING = 0;
+    IN_PROGRESS = 1;
+    SUCCESS = 2;
+    NOT_AVAILABLE = 3;
+  }
+  required StoragePolicySatisfyPathStatus status = 1;
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -1004,4 +1017,6 @@ service ClientNamenodeProtocol {
       returns(SatisfyStoragePolicyResponseProto);
   rpc isStoragePolicySatisfierRunning(IsStoragePolicySatisfierRunningRequestProto)
       returns(IsStoragePolicySatisfierRunningResponseProto);
+  rpc checkStoragePolicySatisfyPathStatus(CheckStoragePolicySatisfyPathStatusRequestProto)
+      returns(CheckStoragePolicySatisfyPathStatusResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index df16f5b..1e579ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -84,6 +85,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Append
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckStoragePolicySatisfyPathStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckStoragePolicySatisfyPathStatusResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
@@ -252,7 +255,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCod
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingCodecsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingCodecsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
@@ -1878,4 +1881,22 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
     return VOID_SATISFYSTORAGEPOLICY_RESPONSE;
   }
+
+  @Override
+  public CheckStoragePolicySatisfyPathStatusResponseProto
+      checkStoragePolicySatisfyPathStatus(RpcController controller,
+      CheckStoragePolicySatisfyPathStatusRequestProto request)
+      throws ServiceException {
+    try {
+      StoragePolicySatisfyPathStatus status = server
+          .checkStoragePolicySatisfyPathStatus(request.getSrc());
+      CheckStoragePolicySatisfyPathStatusResponseProto.Builder builder =
+          CheckStoragePolicySatisfyPathStatusResponseProto
+          .newBuilder();
+      builder.setStatus(PBHelperClient.convert(status));
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/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 2748544..a503c4b 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
@@ -47,6 +47,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
+
 import javax.management.ObjectName;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -68,6 +69,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -5025,4 +5027,14 @@ public class BlockManager implements BlockStatsMXBean {
   public boolean isStoragePolicySatisfierRunning() {
     return sps.isRunning();
   }
+
+  /**
+   * @return status
+   *                Storage policy satisfy status of the path.
+   * @throws IOException
+   */
+  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException {
+    return sps.checkStoragePolicySatisfyPathStatus(path);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
index 788a98b..8f7487c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
@@ -26,13 +26,17 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
 import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,6 +66,9 @@ public class BlockStorageMovementNeeded {
   private final Map<Long, DirPendingWorkInfo> pendingWorkForDirectory =
       new HashMap<Long, DirPendingWorkInfo>();
 
+  private final Map<Long, StoragePolicySatisfyPathStatusInfo> spsStatus =
+      new ConcurrentHashMap<>();
+
   private final Namesystem namesystem;
 
   // List of pending dir to satisfy the policy
@@ -73,6 +80,10 @@ public class BlockStorageMovementNeeded {
 
   private final int maxQueuedItem;
 
+  // Amount of time to cache the SUCCESS status of path before turning it to
+  // NOT_AVAILABLE.
+  private static long statusClearanceElapsedTimeMs = 300000;
+
   public BlockStorageMovementNeeded(Namesystem namesystem,
       StoragePolicySatisfier sps, int queueLimit) {
     this.namesystem = namesystem;
@@ -88,6 +99,9 @@ public class BlockStorageMovementNeeded {
    *          - track info for satisfy the policy
    */
   public synchronized void add(ItemInfo trackInfo) {
+    spsStatus.put(trackInfo.getStartId(),
+        new StoragePolicySatisfyPathStatusInfo(
+            StoragePolicySatisfyPathStatus.IN_PROGRESS));
     storageMovementNeeded.add(trackInfo);
   }
 
@@ -125,6 +139,8 @@ public class BlockStorageMovementNeeded {
   }
 
   public synchronized void addToPendingDirQueue(long id) {
+    spsStatus.put(id, new StoragePolicySatisfyPathStatusInfo(
+        StoragePolicySatisfyPathStatus.PENDING));
     spsDirsToBeTraveresed.add(id);
     // Notify waiting FileInodeIdCollector thread about the newly
     // added SPS path.
@@ -172,6 +188,7 @@ public class BlockStorageMovementNeeded {
       if (inode == null) {
         // directory deleted just remove it.
         this.pendingWorkForDirectory.remove(startId);
+        markSuccess(startId);
       } else {
         DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
         if (pendingWork != null) {
@@ -179,6 +196,7 @@ public class BlockStorageMovementNeeded {
           if (pendingWork.isDirWorkDone()) {
             namesystem.removeXattr(startId, XATTR_SATISFY_STORAGE_POLICY);
             pendingWorkForDirectory.remove(startId);
+            markSuccess(startId);
           }
         }
       }
@@ -187,6 +205,7 @@ public class BlockStorageMovementNeeded {
       // storageMovementAttemptedItems or file policy satisfied.
       namesystem.removeXattr(trackInfo.getTrackId(),
           XATTR_SATISFY_STORAGE_POLICY);
+      markSuccess(trackInfo.getStartId());
     }
   }
 
@@ -203,6 +222,19 @@ public class BlockStorageMovementNeeded {
   }
 
   /**
+   * Mark inode status as SUCCESS in map.
+   */
+  private void markSuccess(long startId){
+    StoragePolicySatisfyPathStatusInfo spsStatusInfo =
+        spsStatus.get(startId);
+    if (spsStatusInfo == null) {
+      spsStatusInfo = new StoragePolicySatisfyPathStatusInfo();
+      spsStatus.put(startId, spsStatusInfo);
+    }
+    spsStatusInfo.setSuccess();
+  }
+
+  /**
    * Clean all the movements in spsDirsToBeTraveresed/storageMovementNeeded
    * and notify to clean up required resources.
    * @throws IOException
@@ -256,6 +288,7 @@ public class BlockStorageMovementNeeded {
     @Override
     public void run() {
       LOG.info("Starting FileInodeIdCollector!.");
+      long lastStatusCleanTime = 0;
       while (namesystem.isRunning() && sps.isRunning()) {
         try {
           if (!namesystem.isInSafeMode()) {
@@ -271,6 +304,9 @@ public class BlockStorageMovementNeeded {
               if (startInode != null) {
                 try {
                   remainingCapacity = remainingCapacity();
+                  spsStatus.put(startINodeId,
+                      new StoragePolicySatisfyPathStatusInfo(
+                          StoragePolicySatisfyPathStatus.IN_PROGRESS));
                   readLock();
                   traverseDir(startInode.asDirectory(), startINodeId,
                       HdfsFileStatus.EMPTY_NAME,
@@ -289,9 +325,16 @@ public class BlockStorageMovementNeeded {
                   namesystem.removeXattr(startInode.getId(),
                       XATTR_SATISFY_STORAGE_POLICY);
                   pendingWorkForDirectory.remove(startInode.getId());
+                  markSuccess(startInode.getId());
                 }
               }
             }
+            //Clear the SPS status if status is in SUCCESS more than 5 min.
+            if (Time.monotonicNow()
+                - lastStatusCleanTime > statusClearanceElapsedTimeMs) {
+              lastStatusCleanTime = Time.monotonicNow();
+              cleanSpsStatus();
+            }
           }
         } catch (Throwable t) {
           LOG.warn("Exception while loading inodes to satisfy the policy", t);
@@ -299,6 +342,16 @@ public class BlockStorageMovementNeeded {
       }
     }
 
+    private synchronized void cleanSpsStatus() {
+      for (Iterator<Entry<Long, StoragePolicySatisfyPathStatusInfo>> it =
+          spsStatus.entrySet().iterator(); it.hasNext();) {
+        Entry<Long, StoragePolicySatisfyPathStatusInfo> entry = it.next();
+        if (entry.getValue().canRemove()) {
+          it.remove();
+        }
+      }
+    }
+
     @Override
     protected void checkPauseForTesting() throws InterruptedException {
       // TODO implement if needed
@@ -434,4 +487,60 @@ public class BlockStorageMovementNeeded {
       return startId;
     }
   }
+
+  /**
+   * Represent the file/directory block movement status.
+   */
+  static class StoragePolicySatisfyPathStatusInfo {
+    private StoragePolicySatisfyPathStatus status =
+        StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
+    private long lastStatusUpdateTime;
+
+    StoragePolicySatisfyPathStatusInfo() {
+      this.lastStatusUpdateTime = 0;
+    }
+
+    StoragePolicySatisfyPathStatusInfo(StoragePolicySatisfyPathStatus status) {
+      this.status = status;
+      this.lastStatusUpdateTime = 0;
+    }
+
+    private void setSuccess() {
+      this.status = StoragePolicySatisfyPathStatus.SUCCESS;
+      this.lastStatusUpdateTime = Time.monotonicNow();
+    }
+
+    private StoragePolicySatisfyPathStatus getStatus() {
+      return status;
+    }
+
+    /**
+     * Return true if SUCCESS status cached more then 5 min.
+     */
+    private boolean canRemove() {
+      return StoragePolicySatisfyPathStatus.SUCCESS == status
+          && (Time.monotonicNow()
+              - lastStatusUpdateTime) > statusClearanceElapsedTimeMs;
+    }
+  }
+
+  public StoragePolicySatisfyPathStatus getStatus(long id) {
+    StoragePolicySatisfyPathStatusInfo spsStatusInfo = spsStatus.get(id);
+    if(spsStatusInfo == null){
+      return StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
+    }
+    return spsStatusInfo.getStatus();
+  }
+
+  @VisibleForTesting
+  public static void setStatusClearanceElapsedTimeMs(
+      long statusClearanceElapsedTimeMs) {
+    BlockStorageMovementNeeded.statusClearanceElapsedTimeMs =
+        statusClearanceElapsedTimeMs;
+  }
+
+  @VisibleForTesting
+  public static long getStatusClearanceElapsedTimeMs() {
+    return statusClearanceElapsedTimeMs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index a938fd3..c52d7ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -28,7 +28,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_DEPTH;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_LENGTH;
-
 import static org.apache.hadoop.util.Time.now;
 
 import java.io.FileNotFoundException;
@@ -109,6 +108,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -2487,4 +2487,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     return namesystem.getBlockManager().isStoragePolicySatisfierRunning();
   }
+
+  @Override
+  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException {
+    checkNNStartup();
+    if (nn.isStandbyState()) {
+      throw new StandbyException("Not supported by Standby Namenode.");
+    }
+    return namesystem.getBlockManager().checkStoragePolicySatisfyPathStatus(
+        path);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/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 cbfba44..2382d36 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.util.Time.monotonicNow;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -36,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.server.balancer.Matcher;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -934,4 +936,10 @@ public class StoragePolicySatisfier implements Runnable {
     }
 
   }
+
+  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException {
+    INode inode = namesystem.getFSDirectory().getINode(path);
+    return storageMovementNeeded.getStatus(inode.getId());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/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 c351410..05498d6 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
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
@@ -258,7 +259,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
 
     @Override
     public String getShortUsage() {
-      return "[" + getName() + " -path <path>]\n";
+      return "[" + getName() + " [-w] -path <path>]\n";
     }
 
     @Override
@@ -266,6 +267,14 @@ public class StoragePolicyAdmin extends Configured implements Tool {
       TableListing listing = AdminHelper.getOptionDescriptionListing();
       listing.addRow("<path>", "The path of the file/directory to satisfy"
           + " storage policy");
+      listing.addRow("-w",
+          "It requests that the command wait till all the files satisfy"
+              + " the policy in given path. This will print the current"
+              + "status of the path in each 10 sec and status are:\n"
+              + "PENDING : Path is in queue and not processed for satisfying"
+              + " the policy.\nIN_PROGRESS : Satisfying the storage policy for"
+              + " path.\nSUCCESS : Storage policy satisfied for the path.\n"
+              + "NOT_AVAILABLE : Status not available.");
       return getShortUsage() + "\n" +
           "Schedule blocks to move based on file/directory policy.\n\n" +
           listing.toString();
@@ -285,12 +294,36 @@ public class StoragePolicyAdmin extends Configured implements Tool {
         dfs.satisfyStoragePolicy(new Path(path));
         System.out.println("Scheduled blocks to move based on the current"
             + " storage policy on " + path);
+        boolean waitOpt = StringUtils.popOption("-w", args);
+        if (waitOpt) {
+          waitForSatisfyPolicy(dfs, path);
+        }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
         return 2;
       }
       return 0;
     }
+
+
+    private void waitForSatisfyPolicy(DistributedFileSystem dfs, String path)
+        throws IOException {
+      System.out.println("Waiting for satisfy the policy ...");
+      while (true) {
+        StoragePolicySatisfyPathStatus status = dfs.getClient()
+            .checkStoragePolicySatisfyPathStatus(path);
+        if (StoragePolicySatisfyPathStatus.SUCCESS.equals(status)) {
+          System.out.println(status);
+          break;
+        }
+        System.out.println(status);
+        try {
+          Thread.sleep(10000);
+        } catch (InterruptedException e) {
+        }
+      }
+      System.out.println(" done");
+    }
   }
 
   /** Command to check storage policy satisfier status. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/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 f69a3ce..93fcb1b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -215,13 +215,14 @@ Schedule blocks to move based on file's/directory's current storage policy.
 
 * Command:
 
-        hdfs storagepolicies -satisfyStoragePolicy -path <path>
+        hdfs storagepolicies -satisfyStoragePolicy [-w] -path <path>
 
 * Arguments:
 
 | | |
 |:---- |:---- |
 | `-path <path>` | The path referring to either a directory or a file. |
+| `-w` | It requests that the command wait till all the files satisfy the policy in given path. This will print the current status of the path in each 10 sec and status are:<br/>PENDING - Path is in queue and not processed for satisfying the policy.<br/>IN_PROGRESS - Satisfying the storage policy for path.<br/>SUCCESS - Storage policy satisfied for the path.<br/>NOT_AVAILABLE - Status not available. |
 
 ### SPS Running Status
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/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 7165d06..c301b8a 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
@@ -479,9 +479,9 @@ public class TestPersistentStoragePolicySatisfier {
       clusterSetUp();
       fs.setStoragePolicy(parentDir, "COLD");
       fs.satisfyStoragePolicy(childDir);
-      fs.satisfyStoragePolicy(parentDir);
       DFSTestUtil.waitExpectedStorageType(childFileName, StorageType.ARCHIVE,
           3, 30000, cluster.getFileSystem());
+      fs.satisfyStoragePolicy(parentDir);
       DFSTestUtil.waitExpectedStorageType(parentFileName, StorageType.ARCHIVE,
           3, 30000, cluster.getFileSystem());
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/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 70219f6..f42d911 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
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -1463,6 +1464,72 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  @Test(timeout = 300000)
+  public void testStoragePolicySatisfyPathStatus() throws Exception {
+    try {
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "3000");
+      config.setBoolean(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY,
+          true);
+
+      StorageType[][] storagetypes = new StorageType[][] {
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK}};
+      hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(2)
+          .storageTypes(storagetypes).build();
+      hdfsCluster.waitActive();
+      BlockStorageMovementNeeded.setStatusClearanceElapsedTimeMs(20000);
+      dfs = hdfsCluster.getFileSystem();
+      Path filePath = new Path("/file");
+      DFSTestUtil.createFile(dfs, filePath, 1024, (short) 2,
+            0);
+      dfs.setStoragePolicy(filePath, "COLD");
+      dfs.satisfyStoragePolicy(filePath);
+      StoragePolicySatisfyPathStatus status = dfs.getClient()
+          .checkStoragePolicySatisfyPathStatus(filePath.toString());
+      Assert.assertTrue("Status should be IN_PROGRESS",
+          StoragePolicySatisfyPathStatus.IN_PROGRESS.equals(status));
+      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+          StorageType.ARCHIVE, 2, 30000, dfs);
+
+      // wait till status is SUCCESS
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            StoragePolicySatisfyPathStatus status = dfs.getClient()
+                .checkStoragePolicySatisfyPathStatus(filePath.toString());
+            return StoragePolicySatisfyPathStatus.SUCCESS.equals(status);
+          } catch (IOException e) {
+            Assert.fail("Fail to get path status for sps");
+          }
+          return false;
+        }
+      }, 100, 60000);
+
+      // wait till status is NOT_AVAILABLE
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            StoragePolicySatisfyPathStatus status = dfs.getClient()
+                .checkStoragePolicySatisfyPathStatus(filePath.toString());
+            return StoragePolicySatisfyPathStatus.NOT_AVAILABLE.equals(status);
+          } catch (IOException e) {
+            Assert.fail("Fail to get path status for sps");
+          }
+          return false;
+        }
+      }, 100, 60000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private static void createDirectoryTree(DistributedFileSystem dfs)
       throws Exception {
     // tree structure

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0e8b0e/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 1a38105..0644a83 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
@@ -204,4 +204,22 @@ public class TestStoragePolicyCommands {
     DFSTestUtil.toolRun(admin, "-isSatisfierRunning status", 1,
         "Can't understand arguments: ");
   }
+
+  @Test(timeout = 90000)
+  public void testSatisfyStoragePolicyCommandWithWaitOption()
+      throws Exception {
+    final String file = "/testSatisfyStoragePolicyCommandWithWaitOption";
+    DFSTestUtil.createFile(fs, new Path(file), SIZE, REPL, 0);
+
+    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
+
+    DFSTestUtil.toolRun(admin, "-setStoragePolicy -path " + file
+        + " -policy COLD", 0, "Set storage policy COLD on " + file.toString());
+
+    DFSTestUtil.toolRun(admin, "-satisfyStoragePolicy -w -path " + file, 0,
+        "Waiting for satisfy the policy");
+
+    DFSTestUtil
+        .waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000, fs);
+  }
 }


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


[10/50] [abbrv] hadoop git commit: HDFS-10802. [SPS]: Add satisfyStoragePolicy API in HdfsAdmin. Contributed by Yuanbo Liu

Posted by ra...@apache.org.
HDFS-10802. [SPS]: Add satisfyStoragePolicy API in HdfsAdmin. Contributed by Yuanbo Liu


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

Branch: refs/heads/HDFS-10285
Commit: 4d2d05309b4e205d679b6c1a4ab648b7d4b464a3
Parents: bfb1a50
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Thu Nov 17 14:07:45 2016 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:20:29 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  19 ++-
 .../hadoop/hdfs/DistributedFileSystem.java      |  34 +++++
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |  10 ++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  14 +++
 .../ClientNamenodeProtocolTranslatorPB.java     |  11 ++
 .../src/main/proto/ClientNamenodeProtocol.proto |  10 ++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  18 +++
 .../server/blockmanagement/BlockManager.java    |   4 +
 .../hdfs/server/namenode/FSDirAttrOp.java       |  56 +++++++++
 .../hdfs/server/namenode/FSNamesystem.java      |  18 +++
 .../hdfs/server/namenode/NameNodeRpcServer.java |   6 +
 .../namenode/TestStoragePolicySatisfier.java    | 124 +++++++++++++++++++
 12 files changed, 323 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 51dff8d..7f204f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -3006,7 +3006,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * has REPLICATION policy.
    * @throws IOException
    */
-
   public ErasureCodingPolicy getErasureCodingPolicy(String src)
       throws IOException {
     checkOpen();
@@ -3019,6 +3018,24 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  /**
+   * Satisfy storage policy for an existing file/directory.
+   * @param src file/directory name
+   * @throws IOException
+   */
+  public void satisfyStoragePolicy(String src) throws IOException {
+    checkOpen();
+    try (TraceScope ignored =
+        newPathTraceScope("satisfyStoragePolicy", src)) {
+      namenode.satisfyStoragePolicy(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          FileNotFoundException.class,
+          SafeModeException.class,
+          UnresolvedPathException.class);
+    }
+  }
+
   Tracer getTracer() {
     return tracer;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 1478868..5ced617 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2665,6 +2665,40 @@ public class DistributedFileSystem extends FileSystem
   }
 
   /**
+   * Set the source path to satisfy storage policy. This API is non-recursive
+   * in nature, i.e., if the source path is a directory then all the files
+   * immediately under the directory would be considered for satisfying the
+   * policy and the sub-directories if any under this path will be skipped.
+   *
+   * @param path The source path referring to either a directory or a file.
+   * @throws IOException
+   */
+  public void satisfyStoragePolicy(final Path path) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+
+      @Override
+      public Void doCall(Path p) throws IOException {
+        dfs.satisfyStoragePolicy(getPathName(p));
+        return null;
+      }
+
+      @Override
+      public Void next(FileSystem fs, Path p) throws IOException {
+        // DFS only
+        if (fs instanceof  DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          myDfs.satisfyStoragePolicy(p);
+          return null;
+        }
+        throw new UnsupportedOperationException(
+            "Cannot satisfyStoragePolicy through a symlink to a "
+                + "non-DistributedFileSystem: " + path + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
    * Get erasure coding policy information for the specified path
    *
    * @param path The path of the file or directory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index 9116167..26293e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
@@ -534,6 +534,16 @@ public class HdfsAdmin {
   }
 
   /**
+   * Set the source path to the specified storage policy.
+   *
+   * @param path The source path referring to either a directory or a file.
+   * @throws IOException
+   */
+  public void satisfyStoragePolicy(final Path path) throws IOException {
+    dfs.satisfyStoragePolicy(path);
+  }
+
+  /**
    * Get the Erasure coding policies supported.
    *
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index eb2e11c..b1affc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1699,4 +1699,18 @@ public interface ClientProtocol {
    */
   @Idempotent
   BatchedEntries<OpenFileEntry> listOpenFiles(long prevId) throws IOException;
+
+  /**
+   * Satisfy the storage policy for a file/directory.
+   * @param path Path of an existing file/directory.
+   * @throws AccessControlException If access is denied.
+   * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>src</code>
+   *           contains a symlink.
+   * @throws java.io.FileNotFoundException If file/dir <code>src</code> is not
+   *           found.
+   * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException append not
+   *           allowed in safemode.
+   */
+  @Idempotent
+  void satisfyStoragePolicy(String path) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 38dc44b..cede194 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -181,6 +181,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Trunca
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
@@ -1888,4 +1889,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Override
+  public void satisfyStoragePolicy(String src) throws IOException {
+    SatisfyStoragePolicyRequestProto req =
+        SatisfyStoragePolicyRequestProto.newBuilder().setSrc(src).build();
+    try {
+      rpcProxy.satisfyStoragePolicy(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index eb6da25..23a5da6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -803,6 +803,14 @@ message ListOpenFilesResponseProto {
   required bool hasMore = 2;
 }
 
+message SatisfyStoragePolicyRequestProto {
+  required string src = 1;
+}
+
+message SatisfyStoragePolicyResponseProto {
+
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -985,4 +993,6 @@ service ClientNamenodeProtocol {
       returns(GetQuotaUsageResponseProto);
   rpc listOpenFiles(ListOpenFilesRequestProto)
       returns(ListOpenFilesResponseProto);
+  rpc satisfyStoragePolicy(SatisfyStoragePolicyRequestProto)
+      returns(SatisfyStoragePolicyResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 2ae41e4..b1f68b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -220,6 +220,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
@@ -398,6 +400,10 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   private static final CheckAccessResponseProto
     VOID_CHECKACCESS_RESPONSE = CheckAccessResponseProto.getDefaultInstance();
 
+  private static final SatisfyStoragePolicyResponseProto
+      VOID_SATISFYSTORAGEPOLICY_RESPONSE = SatisfyStoragePolicyResponseProto
+      .getDefaultInstance();
+
   /**
    * Constructor
    * 
@@ -1842,4 +1848,16 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public SatisfyStoragePolicyResponseProto satisfyStoragePolicy(
+      RpcController controller,
+      SatisfyStoragePolicyRequestProto request) throws ServiceException {
+    try {
+      server.satisfyStoragePolicy(request.getSrc());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_SATISFYSTORAGEPOLICY_RESPONSE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/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 8487a4a..a5edbce 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
@@ -4921,6 +4921,10 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public void satisfyStoragePolicy(long id) {
     storageMovementNeeded.add(id);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Added block collection id {} to block "
+          + "storageMovementNeeded queue", id);
+    }
   }
 
   public StoragePolicySatisfier getStoragePolicySatisfier() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 0dfaa8e..31005d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -197,6 +197,33 @@ public class FSDirAttrOp {
     return fsd.getAuditFileInfo(iip);
   }
 
+  static void satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
+      String src) throws IOException {
+
+    // make sure storage policy is enabled, otherwise
+    // there is no need to satisfy storage policy.
+    if (!fsd.isStoragePolicyEnabled()) {
+      throw new IOException(String.format(
+          "Failed to satisfy storage policy since %s is set to false.",
+          DFS_STORAGE_POLICY_ENABLED_KEY));
+    }
+
+    FSPermissionChecker pc = fsd.getPermissionChecker();
+    INodesInPath iip;
+    fsd.writeLock();
+    try {
+
+      // check operation permission.
+      iip = fsd.resolvePath(pc, src, DirOp.WRITE);
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
+      }
+      unprotectedSatisfyStoragePolicy(bm, iip);
+    } finally {
+      fsd.writeUnlock();
+    }
+  }
+
   static BlockStoragePolicy[] getStoragePolicies(BlockManager bm)
       throws IOException {
     return bm.getStoragePolicies();
@@ -458,6 +485,35 @@ public class FSDirAttrOp {
     }
   }
 
+  static void unprotectedSatisfyStoragePolicy(BlockManager bm,
+      INodesInPath iip) throws IOException {
+
+    // check whether file exists.
+    INode inode = iip.getLastINode();
+    if (inode == null) {
+      throw new FileNotFoundException("File/Directory does not exist: "
+          + iip.getPath());
+    }
+
+    // TODO: need to check whether inode's storage policy
+    // has been satisfied or inode exists in the satisfier
+    // list before calling satisfyStoragePolicy in BlockManager.
+    if (inode.isDirectory()) {
+      final int snapshotId = iip.getLatestSnapshotId();
+      for (INode node : inode.asDirectory().getChildrenList(snapshotId)) {
+        if (node.isFile()) {
+          bm.satisfyStoragePolicy(node.getId());
+
+        }
+      }
+    } else if (inode.isFile()) {
+      bm.satisfyStoragePolicy(inode.getId());
+    } else {
+      throw new FileNotFoundException("File/Directory does not exist: "
+          + iip.getPath());
+    }
+  }
+
   private static void setDirStoragePolicy(
       FSDirectory fsd, INodesInPath iip, byte policyId) throws IOException {
     INode inode = FSDirectory.resolveLastINode(iip);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/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 c524a91..48b60e7 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
@@ -2156,6 +2156,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
+   * Satisfy the storage policy for a file or a directory.
+   *
+   * @param src file/directory path
+   */
+  void satisfyStoragePolicy(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot satisfy storage policy for " + src);
+      // TODO: need to update editlog for persistence.
+      FSDirAttrOp.satisfyStoragePolicy(dir, blockManager, src);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  /**
    * unset storage policy set for a given file or a directory.
    *
    * @param src file/directory path

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 3b325d3..e3173e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1369,6 +1369,12 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
+  public void satisfyStoragePolicy(String src) throws IOException {
+    checkNNStartup();
+    namesystem.satisfyStoragePolicy(src);
+  }
+
+  @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
                        StorageType type)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d2d0530/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 e84052f..901e1ba 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -24,14 +25,18 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -39,6 +44,8 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Supplier;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
+
 /**
  * Tests that StoragePolicySatisfier daemon is able to check the blocks to be
  * moved and finding its suggested target locations to move.
@@ -232,6 +239,123 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for file.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSatisfyFileWithHdfsAdmin() throws Exception {
+    HdfsAdmin hdfsAdmin =
+        new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+    try {
+
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file), "COLD");
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      hdfsAdmin.satisfyStoragePolicy(new Path(file));
+
+      hdfsCluster.triggerHeartbeats();
+      // Wait till namenode notified about the block location details
+      waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy works well for dir.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSatisfyDirWithHdfsAdmin() throws Exception {
+    HdfsAdmin hdfsAdmin =
+        new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+
+    try {
+
+      final String subDir = "/subDir";
+      final String subFile1 = subDir + "/subFile1";
+      final String subDir2 = subDir + "/subDir2";
+      final String subFile2 = subDir2 + "/subFile2";
+      dfs.mkdirs(new Path(subDir));
+      writeContent(subFile1);
+      dfs.mkdirs(new Path(subDir2));
+      writeContent(subFile2);
+
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(subDir), "ONE_SSD");
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      hdfsAdmin.satisfyStoragePolicy(new Path(subDir));
+
+      hdfsCluster.triggerHeartbeats();
+
+      // take effect for the file in the directory.
+      waitExpectedStorageType(subFile1, StorageType.SSD, 1, 30000);
+      waitExpectedStorageType(subFile1, StorageType.DISK, 2, 30000);
+
+      // take no effect for the sub-dir's file in the directory.
+      waitExpectedStorageType(subFile2, StorageType.DEFAULT, 3, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests to verify hdfsAdmin.satisfyStoragePolicy exceptions.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSatisfyWithExceptions() throws Exception {
+    try {
+      final String nonExistingFile = "/noneExistingFile";
+      hdfsCluster.getConfiguration(0).
+          setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
+      hdfsCluster.restartNameNodes();
+      hdfsCluster.waitActive();
+      HdfsAdmin hdfsAdmin =
+          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+
+      try {
+        hdfsAdmin.satisfyStoragePolicy(new Path(file));
+        Assert.fail(String.format(
+            "Should failed to satisfy storage policy "
+                + "for %s since %s is set to false.",
+            file, DFS_STORAGE_POLICY_ENABLED_KEY));
+      } catch (IOException e) {
+        Assert.assertTrue(e.getMessage().contains(String.format(
+            "Failed to satisfy storage policy since %s is set to false.",
+            DFS_STORAGE_POLICY_ENABLED_KEY)));
+      }
+
+      hdfsCluster.getConfiguration(0).
+          setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, true);
+      hdfsCluster.restartNameNodes();
+      hdfsCluster.waitActive();
+      hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(config), config);
+      try {
+        hdfsAdmin.satisfyStoragePolicy(new Path(nonExistingFile));
+        Assert.fail("Should throw FileNotFoundException for " +
+            nonExistingFile);
+      } catch (FileNotFoundException e) {
+
+      }
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
   private void waitForBlocksMovementResult(long expectedBlkMovResultsCount,
       int timeout) throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();


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


[14/50] [abbrv] hadoop git commit: HDFS-11032: [SPS]: Handling of block movement failure at the coordinator datanode. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-11032: [SPS]: Handling of block movement failure at the coordinator datanode. 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/37a4d843
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/37a4d843
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/37a4d843

Branch: refs/heads/HDFS-10285
Commit: 37a4d843d645d52c51453456d37b656c4643b42d
Parents: b19b083
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Thu Dec 22 17:07:49 2016 -0800
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:25 2017 +0530

----------------------------------------------------------------------
 .../datanode/StoragePolicySatisfyWorker.java    |   9 +-
 .../namenode/TestStoragePolicySatisfier.java    | 168 +++++++++++++++----
 2 files changed, 143 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/37a4d843/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index a69a38b..19f3fe2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockPinningException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -251,6 +252,12 @@ public class StoragePolicySatisfyWorker {
                 + " satisfying storageType:{}",
             block, source, target, targetStorageType);
         return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_SUCCESS;
+      } catch (BlockPinningException e) {
+        // Pinned block won't be able to move to a different node. So, its not
+        // required to do retries, just marked as SUCCESS.
+        LOG.debug("Pinned block can't be moved, so skipping block:{}", block,
+            e);
+        return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_SUCCESS;
       } catch (IOException e) {
         // TODO: handle failure retries
         LOG.warn(
@@ -282,7 +289,7 @@ public class StoragePolicySatisfyWorker {
         response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
       }
       String logInfo = "reportedBlock move is failed";
-      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
+      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo, true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37a4d843/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 fe23f3e..179b66b 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
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KE
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
@@ -36,12 +37,15 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -66,10 +70,16 @@ public class TestStoragePolicySatisfier {
   final private long capacity = 2 * 256 * 1024 * 1024;
   final private String file = "/testMoveWhenStoragePolicyNotSatisfying";
   private DistributedFileSystem dfs = null;
+  private static final int DEFAULT_BLOCK_SIZE = 1024;
 
-  @Before
-  public void setUp() throws IOException {
-    config.setLong("dfs.block.size", 1024);
+  private void shutdownCluster() {
+    if (hdfsCluster != null) {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  private void createCluster() throws IOException {
+    config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     hdfsCluster = startCluster(config, allDiskTypes, numOfDatanodes,
         storagesPerDatanode, capacity);
     dfs = hdfsCluster.getFileSystem();
@@ -81,6 +91,7 @@ public class TestStoragePolicySatisfier {
       throws Exception {
 
     try {
+      createCluster();
       // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), "COLD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
@@ -99,7 +110,7 @@ public class TestStoragePolicySatisfier {
       // Wait till namenode notified about the block location details
       waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -107,6 +118,7 @@ public class TestStoragePolicySatisfier {
   public void testWhenStoragePolicySetToALLSSD()
       throws Exception {
     try {
+      createCluster();
       // Change policy to ALL_SSD
       dfs.setStoragePolicy(new Path(file), "ALL_SSD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
@@ -127,7 +139,7 @@ public class TestStoragePolicySatisfier {
       // areas
       waitExpectedStorageType(file, StorageType.SSD, 3, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -135,6 +147,7 @@ public class TestStoragePolicySatisfier {
   public void testWhenStoragePolicySetToONESSD()
       throws Exception {
     try {
+      createCluster();
       // Change policy to ONE_SSD
       dfs.setStoragePolicy(new Path(file), "ONE_SSD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
@@ -154,7 +167,7 @@ public class TestStoragePolicySatisfier {
       waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
       waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -165,6 +178,7 @@ public class TestStoragePolicySatisfier {
   @Test(timeout = 300000)
   public void testPerTrackIdBlocksStorageMovementResults() throws Exception {
     try {
+      createCluster();
       // Change policy to ONE_SSD
       dfs.setStoragePolicy(new Path(file), "ONE_SSD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
@@ -186,7 +200,7 @@ public class TestStoragePolicySatisfier {
 
       waitForBlocksMovementResult(1, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -196,18 +210,18 @@ public class TestStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testMultipleFilesForSatisfyStoragePolicy() throws Exception {
-    List<String> files = new ArrayList<>();
-    files.add(file);
-
-    // Creates 4 more files. Send all of them for satisfying the storage policy
-    // together.
-    for (int i = 0; i < 4; i++) {
-      String file1 = "/testMoveWhenStoragePolicyNotSatisfying_" + i;
-      files.add(file1);
-      writeContent(file1);
-    }
-
     try {
+      createCluster();
+      List<String> files = new ArrayList<>();
+      files.add(file);
+
+      // Creates 4 more files. Send all of them for satisfying the storage
+      // policy together.
+      for (int i = 0; i < 4; i++) {
+        String file1 = "/testMoveWhenStoragePolicyNotSatisfying_" + i;
+        files.add(file1);
+        writeContent(file1);
+      }
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       List<Long> blockCollectionIds = new ArrayList<>();
       // Change policy to ONE_SSD
@@ -237,7 +251,7 @@ public class TestStoragePolicySatisfier {
 
       waitForBlocksMovementResult(blockCollectionIds.size(), 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -247,10 +261,10 @@ public class TestStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testSatisfyFileWithHdfsAdmin() throws Exception {
-    HdfsAdmin hdfsAdmin =
-        new HdfsAdmin(FileSystem.getDefaultUri(config), config);
     try {
-
+      createCluster();
+      HdfsAdmin hdfsAdmin =
+          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
       // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), "COLD");
 
@@ -267,7 +281,7 @@ public class TestStoragePolicySatisfier {
       // Wait till namenode notified about the block location details
       waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -277,11 +291,10 @@ public class TestStoragePolicySatisfier {
    */
   @Test(timeout = 300000)
   public void testSatisfyDirWithHdfsAdmin() throws Exception {
-    HdfsAdmin hdfsAdmin =
-        new HdfsAdmin(FileSystem.getDefaultUri(config), config);
-
     try {
-
+      createCluster();
+      HdfsAdmin hdfsAdmin =
+          new HdfsAdmin(FileSystem.getDefaultUri(config), config);
       final String subDir = "/subDir";
       final String subFile1 = subDir + "/subFile1";
       final String subDir2 = subDir + "/subDir2";
@@ -310,7 +323,7 @@ public class TestStoragePolicySatisfier {
       // take no effect for the sub-dir's file in the directory.
       waitExpectedStorageType(subFile2, StorageType.DEFAULT, 3, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -321,6 +334,7 @@ public class TestStoragePolicySatisfier {
   @Test(timeout = 300000)
   public void testSatisfyWithExceptions() throws Exception {
     try {
+      createCluster();
       final String nonExistingFile = "/noneExistingFile";
       hdfsCluster.getConfiguration(0).
           setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
@@ -354,7 +368,7 @@ public class TestStoragePolicySatisfier {
 
       }
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -376,6 +390,7 @@ public class TestStoragePolicySatisfier {
   public void testWhenOnlyFewTargetDatanodeAreAvailableToSatisfyStoragePolicy()
       throws Exception {
     try {
+      createCluster();
       // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), "COLD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
@@ -397,7 +412,7 @@ public class TestStoragePolicySatisfier {
 
       waitForBlocksMovementResult(1, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -418,6 +433,7 @@ public class TestStoragePolicySatisfier {
   public void testWhenNoTargetDatanodeToSatisfyStoragePolicy()
       throws Exception {
     try {
+      createCluster();
       // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), "COLD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
@@ -440,7 +456,7 @@ public class TestStoragePolicySatisfier {
       // re-attempted.
       waitForAttemptedItems(1, 30000);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
     }
   }
 
@@ -452,6 +468,7 @@ public class TestStoragePolicySatisfier {
   public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
       throws IOException {
     try {
+      createCluster();
       // Simulate Mover by creating MOVER_ID file
       DFSTestUtil.createFile(hdfsCluster.getFileSystem(),
           HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
@@ -461,8 +478,93 @@ public class TestStoragePolicySatisfier {
       Assert.assertFalse("SPS should not start "
           + "when a Mover instance is running", running);
     } finally {
-      hdfsCluster.shutdown();
+      shutdownCluster();
+    }
+  }
+
+  /**
+   * Test to verify that satisfy worker can't move blocks. If the given block is
+   * pinned it shouldn't be considered for retries.
+   */
+  @Test(timeout = 120000)
+  public void testMoveWithBlockPinning() throws Exception {
+    config.setBoolean(DFSConfigKeys.DFS_DATANODE_BLOCK_PINNING_ENABLED, true);
+    hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(3)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.DISK},
+                {StorageType.DISK, StorageType.DISK},
+                {StorageType.DISK, StorageType.DISK}})
+        .build();
+
+    hdfsCluster.waitActive();
+    dfs = hdfsCluster.getFileSystem();
+
+    // create a file with replication factor 3 and mark 2 pinned block
+    // locations.
+    final String file1 = createFileAndSimulateFavoredNodes(2);
+
+    // Change policy to COLD
+    dfs.setStoragePolicy(new Path(file1), "COLD");
+    FSNamesystem namesystem = hdfsCluster.getNamesystem();
+    INode inode = namesystem.getFSDirectory().getINode(file1);
+
+    StorageType[][] newtypes =
+        new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
+            {StorageType.ARCHIVE, StorageType.ARCHIVE},
+            {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+    // Adding DISK based datanodes
+    startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+        storagesPerDatanode, capacity, hdfsCluster);
+
+    namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+    hdfsCluster.triggerHeartbeats();
+
+    // No block movement will be scheduled as there is no target node available
+    // with the required storage type.
+    waitForAttemptedItems(1, 30000);
+    waitForBlocksMovementResult(1, 30000);
+    waitExpectedStorageType(file1, StorageType.ARCHIVE, 1, 30000);
+    waitExpectedStorageType(file1, StorageType.DISK, 2, 30000);
+  }
+
+  private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
+      throws IOException {
+    ArrayList<DataNode> dns = hdfsCluster.getDataNodes();
+    final String file1 = "/testMoveWithBlockPinning";
+    // replication factor 3
+    InetSocketAddress[] favoredNodes = new InetSocketAddress[favoredNodesCount];
+    for (int i = 0; i < favoredNodesCount; i++) {
+      favoredNodes[i] = dns.get(i).getXferAddress();
+    }
+    DFSTestUtil.createFile(dfs, new Path(file1), false, 1024, 100,
+        DEFAULT_BLOCK_SIZE, (short) 3, 0, false, favoredNodes);
+
+    LocatedBlocks locatedBlocks = dfs.getClient().getLocatedBlocks(file1, 0);
+    Assert.assertEquals("Wrong block count", 1,
+        locatedBlocks.locatedBlockCount());
+
+    // verify storage type before movement
+    LocatedBlock lb = locatedBlocks.get(0);
+    StorageType[] storageTypes = lb.getStorageTypes();
+    for (StorageType storageType : storageTypes) {
+      Assert.assertTrue(StorageType.DISK == storageType);
+    }
+
+    // Mock FsDatasetSpi#getPinning to show that the block is pinned.
+    DatanodeInfo[] locations = lb.getLocations();
+    Assert.assertEquals(3, locations.length);
+    Assert.assertTrue(favoredNodesCount < locations.length);
+    for(DatanodeInfo dnInfo: locations){
+      LOG.info("Simulate block pinning in datanode {}",
+          locations[favoredNodesCount]);
+      DataNode dn = hdfsCluster.getDataNode(dnInfo.getIpcPort());
+      DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
+      favoredNodesCount--;
+      if (favoredNodesCount <= 0) {
+        break;// marked favoredNodesCount number of pinned block location
+      }
     }
+    return file1;
   }
 
   private void waitForAttemptedItems(long expectedBlkMovAttemptedCount,


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


[46/50] [abbrv] hadoop git commit: HDFS-12570: [SPS]: Refactor Co-ordinator datanode logic to track the block storage movements. Contributed by Rakesh R.

Posted by ra...@apache.org.
HDFS-12570: [SPS]: Refactor Co-ordinator datanode logic to track the block storage movements. 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/42b22d97
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/42b22d97
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/42b22d97

Branch: refs/heads/HDFS-10285
Commit: 42b22d978176391d98007cd30043ba1b6893fa82
Parents: e64a428
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Thu Oct 12 17:17:51 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:31:13 2017 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |  12 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   4 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 150 +++-----
 .../blockmanagement/DatanodeDescriptor.java     |  50 ++-
 .../server/blockmanagement/DatanodeManager.java | 104 ++++--
 .../hdfs/server/datanode/BPOfferService.java    |   3 +-
 .../hdfs/server/datanode/BPServiceActor.java    |  33 +-
 .../datanode/BlockStorageMovementTracker.java   |  80 ++---
 .../datanode/StoragePolicySatisfyWorker.java    | 214 ++++--------
 .../BlockStorageMovementAttemptedItems.java     | 299 ++++------------
 .../BlockStorageMovementInfosBatch.java         |  61 ----
 .../hdfs/server/namenode/FSNamesystem.java      |  11 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   7 +-
 .../server/namenode/StoragePolicySatisfier.java | 343 ++++++++++---------
 .../protocol/BlockStorageMovementCommand.java   |  99 ++----
 .../BlocksStorageMoveAttemptFinished.java       |  48 +++
 .../protocol/BlocksStorageMovementResult.java   |  74 ----
 .../hdfs/server/protocol/DatanodeProtocol.java  |   5 +-
 .../src/main/proto/DatanodeProtocol.proto       |  30 +-
 .../src/main/resources/hdfs-default.xml         |  21 +-
 .../src/site/markdown/ArchivalStorage.md        |   6 +-
 .../TestNameNodePrunesMissingStorages.java      |   5 +-
 .../datanode/InternalDataNodeTestUtils.java     |   4 +-
 .../server/datanode/TestBPOfferService.java     |   4 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   4 +-
 .../server/datanode/TestDataNodeLifeline.java   |   6 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   4 +-
 .../server/datanode/TestFsDatasetCache.java     |   4 +-
 .../TestStoragePolicySatisfyWorker.java         |  52 ++-
 .../hdfs/server/datanode/TestStorageReport.java |   4 +-
 .../server/namenode/NNThroughputBenchmark.java  |   6 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |   4 +-
 .../TestBlockStorageMovementAttemptedItems.java | 145 ++++----
 .../hdfs/server/namenode/TestDeadDatanode.java  |   4 +-
 .../namenode/TestStoragePolicySatisfier.java    | 115 ++++++-
 ...stStoragePolicySatisfierWithStripedFile.java |  20 +-
 37 files changed, 908 insertions(+), 1135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 a73d399..a7a97a9 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
@@ -585,11 +585,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   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 =
-      5 * 60 * 1000;
+      1 * 60 * 1000;
   public static final String DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY =
       "dfs.storage.policy.satisfier.self.retry.timeout.millis";
   public static final int DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT =
-      20 * 60 * 1000;
+      5 * 60 * 1000;
+  public static final String DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY =
+      "dfs.storage.policy.satisfier.low.max-streams.preference";
+  public static final boolean DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_DEFAULT =
+      false;
 
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 9866;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 9dd87d0..dcc0705 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlock
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -140,7 +140,8 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
       @Nonnull SlowDiskReports slowDisks,
-      BlocksStorageMovementResult[] blksMovementResults) throws IOException {
+      BlocksStorageMoveAttemptFinished storageMovementFinishedBlks)
+          throws IOException {
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
@@ -165,8 +166,11 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     }
 
     // Adding blocks movement results to the heart beat request.
-    builder.addAllBlksMovementResults(
-        PBHelper.convertBlksMovResults(blksMovementResults));
+    if (storageMovementFinishedBlks != null
+        && storageMovementFinishedBlks.getBlocks() != null) {
+      builder.setStorageMoveAttemptFinishedBlks(
+          PBHelper.convertBlksMovReport(storageMovementFinishedBlks));
+    }
 
     HeartbeatResponseProto resp;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 40458ef..b5bb80a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -123,8 +123,8 @@ public class DatanodeProtocolServerSideTranslatorPB implements
           volumeFailureSummary, request.getRequestFullBlockReportLease(),
           PBHelper.convertSlowPeerInfo(request.getSlowPeersList()),
           PBHelper.convertSlowDiskInfo(request.getSlowDisksList()),
-          PBHelper.convertBlksMovResults(
-              request.getBlksMovementResultsList()));
+          PBHelper.convertBlksMovReport(
+              request.getStorageMoveAttemptFinishedBlks()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 5044c0b..d329f9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBand
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECReconstructionCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockMovingInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
@@ -54,11 +55,11 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SlowPeerRepo
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockStorageMovementCommandProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockStorageMovementProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlocksStorageMovementResultProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlocksStorageMoveAttemptFinishedProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstructionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
@@ -100,8 +101,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStr
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult.Status;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
@@ -967,59 +967,27 @@ public class PBHelper {
     return SlowDiskReports.create(slowDisksMap);
   }
 
-  public static BlocksStorageMovementResult[] convertBlksMovResults(
-      List<BlocksStorageMovementResultProto> protos) {
-    BlocksStorageMovementResult[] results =
-        new BlocksStorageMovementResult[protos.size()];
-    for (int i = 0; i < protos.size(); i++) {
-      BlocksStorageMovementResultProto resultProto = protos.get(i);
-      BlocksStorageMovementResult.Status status;
-      switch (resultProto.getStatus()) {
-      case SUCCESS:
-        status = Status.SUCCESS;
-        break;
-      case FAILURE:
-        status = Status.FAILURE;
-        break;
-      case IN_PROGRESS:
-        status = Status.IN_PROGRESS;
-        break;
-      default:
-        throw new AssertionError("Unknown status: " + resultProto.getStatus());
-      }
-      results[i] = new BlocksStorageMovementResult(resultProto.getTrackID(),
-          status);
+  public static BlocksStorageMoveAttemptFinished convertBlksMovReport(
+      BlocksStorageMoveAttemptFinishedProto proto) {
+
+    List<BlockProto> blocksList = proto.getBlocksList();
+    Block[] blocks = new Block[blocksList.size()];
+    for (int i = 0; i < blocksList.size(); i++) {
+      BlockProto blkProto = blocksList.get(i);
+      blocks[i] = PBHelperClient.convert(blkProto);
     }
-    return results;
+    return new BlocksStorageMoveAttemptFinished(blocks);
   }
 
-  public static List<BlocksStorageMovementResultProto> convertBlksMovResults(
-      BlocksStorageMovementResult[] blocksMovementResults) {
-    List<BlocksStorageMovementResultProto> blocksMovementResultsProto =
-        new ArrayList<>();
-    BlocksStorageMovementResultProto.Builder builder =
-        BlocksStorageMovementResultProto.newBuilder();
-    for (int i = 0; i < blocksMovementResults.length; i++) {
-      BlocksStorageMovementResult report = blocksMovementResults[i];
-      builder.setTrackID(report.getTrackId());
-      BlocksStorageMovementResultProto.Status status;
-      switch (report.getStatus()) {
-      case SUCCESS:
-        status = BlocksStorageMovementResultProto.Status.SUCCESS;
-        break;
-      case FAILURE:
-        status = BlocksStorageMovementResultProto.Status.FAILURE;
-        break;
-      case IN_PROGRESS:
-        status = BlocksStorageMovementResultProto.Status.IN_PROGRESS;
-        break;
-      default:
-        throw new AssertionError("Unknown status: " + report.getStatus());
-      }
-      builder.setStatus(status);
-      blocksMovementResultsProto.add(builder.build());
+  public static BlocksStorageMoveAttemptFinishedProto convertBlksMovReport(
+      BlocksStorageMoveAttemptFinished blocksMoveAttemptFinished) {
+    BlocksStorageMoveAttemptFinishedProto.Builder builder =
+        BlocksStorageMoveAttemptFinishedProto.newBuilder();
+    Block[] blocks = blocksMoveAttemptFinished.getBlocks();
+    for (Block block : blocks) {
+      builder.addBlocks(PBHelperClient.convert(block));
     }
-    return blocksMovementResultsProto;
+    return builder.build();
   }
 
   public static JournalInfo convert(JournalInfoProto info) {
@@ -1183,34 +1151,34 @@ public class PBHelper {
     BlockStorageMovementCommandProto.Builder builder =
         BlockStorageMovementCommandProto.newBuilder();
 
-    builder.setTrackID(blkStorageMovementCmd.getTrackID());
     builder.setBlockPoolId(blkStorageMovementCmd.getBlockPoolId());
     Collection<BlockMovingInfo> blockMovingInfos = blkStorageMovementCmd
         .getBlockMovingTasks();
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      builder.addBlockStorageMovement(
-          convertBlockMovingInfo(blkMovingInfo));
+      builder.addBlockMovingInfo(convertBlockMovingInfo(blkMovingInfo));
     }
     return builder.build();
   }
 
-  private static BlockStorageMovementProto convertBlockMovingInfo(
+  private static BlockMovingInfoProto convertBlockMovingInfo(
       BlockMovingInfo blkMovingInfo) {
-    BlockStorageMovementProto.Builder builder = BlockStorageMovementProto
+    BlockMovingInfoProto.Builder builder = BlockMovingInfoProto
         .newBuilder();
     builder.setBlock(PBHelperClient.convert(blkMovingInfo.getBlock()));
 
-    DatanodeInfo[] sourceDnInfos = blkMovingInfo.getSources();
-    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
+    DatanodeInfo sourceDnInfo = blkMovingInfo.getSource();
+    builder.setSourceDnInfo(PBHelperClient.convert(sourceDnInfo));
 
-    DatanodeInfo[] targetDnInfos = blkMovingInfo.getTargets();
-    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
+    DatanodeInfo targetDnInfo = blkMovingInfo.getTarget();
+    builder.setTargetDnInfo(PBHelperClient.convert(targetDnInfo));
 
-    StorageType[] sourceStorageTypes = blkMovingInfo.getSourceStorageTypes();
-    builder.setSourceStorageTypes(convertStorageTypesProto(sourceStorageTypes));
+    StorageType sourceStorageType = blkMovingInfo.getSourceStorageType();
+    builder.setSourceStorageType(
+        PBHelperClient.convertStorageType(sourceStorageType));
 
-    StorageType[] targetStorageTypes = blkMovingInfo.getTargetStorageTypes();
-    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
+    StorageType targetStorageType = blkMovingInfo.getTargetStorageType();
+    builder.setTargetStorageType(
+        PBHelperClient.convertStorageType(targetStorageType));
 
     return builder.build();
   }
@@ -1218,42 +1186,38 @@ public class PBHelper {
   private static DatanodeCommand convert(
       BlockStorageMovementCommandProto blkStorageMovementCmdProto) {
     Collection<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
-    List<BlockStorageMovementProto> blkSPSatisfyList =
-        blkStorageMovementCmdProto.getBlockStorageMovementList();
-    for (BlockStorageMovementProto blkSPSatisfy : blkSPSatisfyList) {
+    List<BlockMovingInfoProto> blkSPSatisfyList =
+        blkStorageMovementCmdProto.getBlockMovingInfoList();
+    for (BlockMovingInfoProto blkSPSatisfy : blkSPSatisfyList) {
       blockMovingInfos.add(convertBlockMovingInfo(blkSPSatisfy));
     }
     return new BlockStorageMovementCommand(
         DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT,
-        blkStorageMovementCmdProto.getTrackID(),
         blkStorageMovementCmdProto.getBlockPoolId(), blockMovingInfos);
   }
 
   private static BlockMovingInfo convertBlockMovingInfo(
-      BlockStorageMovementProto blockStoragePolicySatisfyProto) {
-    BlockProto blockProto = blockStoragePolicySatisfyProto.getBlock();
+      BlockMovingInfoProto blockStorageMovingInfoProto) {
+    BlockProto blockProto = blockStorageMovingInfoProto.getBlock();
     Block block = PBHelperClient.convert(blockProto);
 
-    DatanodeInfosProto sourceDnInfosProto = blockStoragePolicySatisfyProto
-        .getSourceDnInfos();
-    DatanodeInfo[] sourceDnInfos = PBHelperClient.convert(sourceDnInfosProto);
-
-    DatanodeInfosProto targetDnInfosProto = blockStoragePolicySatisfyProto
-        .getTargetDnInfos();
-    DatanodeInfo[] targetDnInfos = PBHelperClient.convert(targetDnInfosProto);
-
-    StorageTypesProto srcStorageTypesProto = blockStoragePolicySatisfyProto
-        .getSourceStorageTypes();
-    StorageType[] srcStorageTypes = PBHelperClient.convertStorageTypes(
-        srcStorageTypesProto.getStorageTypesList(),
-        srcStorageTypesProto.getStorageTypesList().size());
-
-    StorageTypesProto targetStorageTypesProto = blockStoragePolicySatisfyProto
-        .getTargetStorageTypes();
-    StorageType[] targetStorageTypes = PBHelperClient.convertStorageTypes(
-        targetStorageTypesProto.getStorageTypesList(),
-        targetStorageTypesProto.getStorageTypesList().size());
-    return new BlockMovingInfo(block, sourceDnInfos, targetDnInfos,
-        srcStorageTypes, targetStorageTypes);
+    DatanodeInfoProto sourceDnInfoProto = blockStorageMovingInfoProto
+        .getSourceDnInfo();
+    DatanodeInfo sourceDnInfo = PBHelperClient.convert(sourceDnInfoProto);
+
+    DatanodeInfoProto targetDnInfoProto = blockStorageMovingInfoProto
+        .getTargetDnInfo();
+    DatanodeInfo targetDnInfo = PBHelperClient.convert(targetDnInfoProto);
+    StorageTypeProto srcStorageTypeProto = blockStorageMovingInfoProto
+        .getSourceStorageType();
+    StorageType srcStorageType = PBHelperClient
+        .convertStorageType(srcStorageTypeProto);
+
+    StorageTypeProto targetStorageTypeProto = blockStorageMovingInfoProto
+        .getTargetStorageType();
+    StorageType targetStorageType = PBHelperClient
+        .convertStorageType(targetStorageTypeProto);
+    return new BlockMovingInfo(block, sourceDnInfo, targetDnInfo,
+        srcStorageType, targetStorageType);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 95cb3a5..9210e59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.namenode.BlockStorageMovementInfosBatch;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
@@ -211,7 +210,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * A queue of blocks corresponding to trackID for moving its storage
    * placements by this datanode.
    */
-  private final Queue<BlockStorageMovementInfosBatch> storageMovementBlocks =
+  private final Queue<BlockMovingInfo> storageMovementBlocks =
       new LinkedList<>();
   private volatile boolean dropSPSWork = false;
 
@@ -1029,30 +1028,45 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Add the block infos which needs to move its storage locations.
    *
-   * @param trackID
-   *          - unique identifier which will be used for tracking the given set
-   *          of blocks movement completion.
-   * @param storageMismatchedBlocks
-   *          - storage mismatched block infos
+   * @param blkMovingInfo
+   *          - storage mismatched block info
    */
-  public void addBlocksToMoveStorage(long trackID,
-      List<BlockMovingInfo> storageMismatchedBlocks) {
+  public void addBlocksToMoveStorage(BlockMovingInfo blkMovingInfo) {
     synchronized (storageMovementBlocks) {
-      storageMovementBlocks.offer(
-          new BlockStorageMovementInfosBatch(trackID, storageMismatchedBlocks));
+      storageMovementBlocks.offer(blkMovingInfo);
     }
   }
 
   /**
-   * @return block infos which needs to move its storage locations. This returns
-   *         list of blocks under one trackId.
+   * Return the number of blocks queued up for movement.
    */
-  public BlockStorageMovementInfosBatch getBlocksToMoveStorages() {
+  public int getNumberOfBlocksToMoveStorages() {
+    return storageMovementBlocks.size();
+  }
+
+  /**
+   * Get the blocks to move to satisfy the storage media type.
+   *
+   * @param numBlocksToMoveTasks
+   *          total number of blocks which will be send to this datanode for
+   *          block movement.
+   *
+   * @return block infos which needs to move its storage locations.
+   */
+  public BlockMovingInfo[] getBlocksToMoveStorages(int numBlocksToMoveTasks) {
     synchronized (storageMovementBlocks) {
-      // TODO: Presently returning the list of blocks under one trackId.
-      // Need to limit the list of items into small batches with in trackId
-      // itself if blocks are many(For example: a file contains many blocks).
-      return storageMovementBlocks.poll();
+      List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+      for (; !storageMovementBlocks.isEmpty()
+          && numBlocksToMoveTasks > 0; numBlocksToMoveTasks--) {
+        blockMovingInfos.add(storageMovementBlocks.poll());
+      }
+      BlockMovingInfo[] blkMoveArray = new BlockMovingInfo[blockMovingInfos
+          .size()];
+      blkMoveArray = blockMovingInfos.toArray(blkMoveArray);
+      if (blkMoveArray.length > 0) {
+        return blkMoveArray;
+      }
+      return null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 3504cb0..4ea41d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.common.Util;
-import org.apache.hadoop.hdfs.server.namenode.BlockStorageMovementInfosBatch;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
@@ -48,6 +47,7 @@ import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
@@ -208,6 +208,8 @@ public class DatanodeManager {
    */
   private final long timeBetweenResendingCachingDirectivesMs;
 
+  private final boolean blocksToMoveShareEqualRatio;
+
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
@@ -332,6 +334,12 @@ public class DatanodeManager {
     this.blocksPerPostponedMisreplicatedBlocksRescan = conf.getLong(
         DFSConfigKeys.DFS_NAMENODE_BLOCKS_PER_POSTPONEDBLOCKS_RESCAN_KEY,
         DFSConfigKeys.DFS_NAMENODE_BLOCKS_PER_POSTPONEDBLOCKS_RESCAN_KEY_DEFAULT);
+
+    // SPS configuration to decide blocks to move can share equal ratio of
+    // maxtransfers with pending replica and erasure-coded reconstruction tasks
+    blocksToMoveShareEqualRatio = conf.getBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_DEFAULT);
   }
 
   private static long getStaleIntervalFromConf(Configuration conf,
@@ -1092,13 +1100,14 @@ public class DatanodeManager {
           // Sets dropSPSWork flag to true, to ensure that
           // DNA_DROP_SPS_WORK_COMMAND will send to datanode via next heartbeat
           // response immediately after the node registration. This is
-          // to avoid a situation, where multiple trackId responses coming from
-          // different co-odinator datanodes. After SPS monitor time out, it
-          // will retry the files which were scheduled to the disconnected(for
-          // long time more than heartbeat expiry) DN, by finding new
-          // co-ordinator datanode. Now, if the expired datanode reconnects back
-          // after SPS reschedules, it leads to get different movement results
-          // from reconnected and new DN co-ordinators.
+          // to avoid a situation, where multiple block attempt finished
+          // responses coming from different datanodes. After SPS monitor time
+          // out, it will retry the files which were scheduled to the
+          // disconnected(for long time more than heartbeat expiry) DN, by
+          // finding new datanode. Now, if the expired datanode reconnects back
+          // after SPS reschedules, it leads to get different movement attempt
+          // finished report from reconnected and newly datanode which is
+          // attempting the block movement.
           nodeS.setDropSPSWork(true);
 
           // resolve network location
@@ -1678,19 +1687,47 @@ public class DatanodeManager {
     final List<DatanodeCommand> cmds = new ArrayList<>();
     // Allocate _approximately_ maxTransfers pending tasks to DataNode.
     // NN chooses pending tasks based on the ratio between the lengths of
-    // replication and erasure-coded block queues.
+    // replication, erasure-coded block queues and block storage movement
+    // queues.
     int totalReplicateBlocks = nodeinfo.getNumberOfReplicateBlocks();
     int totalECBlocks = nodeinfo.getNumberOfBlocksToBeErasureCoded();
+    int totalBlocksToMove = nodeinfo.getNumberOfBlocksToMoveStorages();
     int totalBlocks = totalReplicateBlocks + totalECBlocks;
-    if (totalBlocks > 0) {
-      int numReplicationTasks = (int) Math.ceil(
-          (double) (totalReplicateBlocks * maxTransfers) / totalBlocks);
-      int numECTasks = (int) Math.ceil(
-          (double) (totalECBlocks * maxTransfers) / totalBlocks);
-
+    if (totalBlocks > 0 || totalBlocksToMove > 0) {
+      int numReplicationTasks = 0;
+      int numECTasks = 0;
+      int numBlocksToMoveTasks = 0;
+      // Check blocksToMoveShareEqualRatio configuration is true/false. If true,
+      // then equally sharing the max transfer. Otherwise gives high priority to
+      // the pending_replica/erasure-coded tasks and only the delta streams will
+      // be used for blocks to move tasks.
+      if (blocksToMoveShareEqualRatio) {
+        // add blocksToMove count to total blocks so that will get equal share
+        totalBlocks = totalBlocks + totalBlocksToMove;
+        numReplicationTasks = (int) Math
+            .ceil((double) (totalReplicateBlocks * maxTransfers) / totalBlocks);
+        numECTasks = (int) Math
+            .ceil((double) (totalECBlocks * maxTransfers) / totalBlocks);
+        numBlocksToMoveTasks = (int) Math
+            .ceil((double) (totalBlocksToMove * maxTransfers) / totalBlocks);
+      } else {
+        // Calculate the replica and ec tasks, then pick blocksToMove if there
+        // is any streams available.
+        numReplicationTasks = (int) Math
+            .ceil((double) (totalReplicateBlocks * maxTransfers) / totalBlocks);
+        numECTasks = (int) Math
+            .ceil((double) (totalECBlocks * maxTransfers) / totalBlocks);
+        int numTasks = numReplicationTasks + numECTasks;
+        if (numTasks < maxTransfers) {
+          int remainingMaxTransfers = maxTransfers - numTasks;
+          numBlocksToMoveTasks = Math.min(totalBlocksToMove,
+              remainingMaxTransfers);
+        }
+      }
       if (LOG.isDebugEnabled()) {
         LOG.debug("Pending replication tasks: " + numReplicationTasks
-            + " erasure-coded tasks: " + numECTasks);
+            + " erasure-coded tasks: " + numECTasks + " blocks to move tasks: "
+            + numBlocksToMoveTasks);
       }
       // check pending replication tasks
       List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
@@ -1706,6 +1743,23 @@ public class DatanodeManager {
         cmds.add(new BlockECReconstructionCommand(
             DNA_ERASURE_CODING_RECONSTRUCTION, pendingECList));
       }
+      // check pending block storage movement tasks
+      if (nodeinfo.shouldDropSPSWork()) {
+        cmds.add(DropSPSWorkCommand.DNA_DROP_SPS_WORK_COMMAND);
+        // Set back to false to indicate that the new value has been sent to the
+        // datanode.
+        nodeinfo.setDropSPSWork(false);
+      } else {
+        // Get pending block storage movement tasks
+        BlockMovingInfo[] blkStorageMovementInfos = nodeinfo
+            .getBlocksToMoveStorages(numBlocksToMoveTasks);
+
+        if (blkStorageMovementInfos != null) {
+          cmds.add(new BlockStorageMovementCommand(
+              DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT, blockPoolId,
+              Arrays.asList(blkStorageMovementInfos)));
+        }
+      }
     }
 
     // check block invalidation
@@ -1749,24 +1803,6 @@ public class DatanodeManager {
       }
     }
 
-    if (nodeinfo.shouldDropSPSWork()) {
-      cmds.add(DropSPSWorkCommand.DNA_DROP_SPS_WORK_COMMAND);
-      // Set back to false to indicate that the new value has been sent to the
-      // datanode.
-      nodeinfo.setDropSPSWork(false);
-    }
-
-    // check pending block storage movement tasks
-    BlockStorageMovementInfosBatch blkStorageMovementInfosBatch = nodeinfo
-        .getBlocksToMoveStorages();
-
-    if (blkStorageMovementInfosBatch != null) {
-      cmds.add(new BlockStorageMovementCommand(
-          DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT,
-          blkStorageMovementInfosBatch.getTrackID(), blockPoolId,
-          blkStorageMovementInfosBatch.getBlockMovingInfo()));
-    }
-
     if (!cmds.isEmpty()) {
       return cmds.toArray(new DatanodeCommand[cmds.size()]);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index d60fb6d..65ab246 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -785,8 +785,7 @@ class BPOfferService {
       LOG.info("DatanodeCommand action: DNA_BLOCK_STORAGE_MOVEMENT");
       BlockStorageMovementCommand blkSPSCmd = (BlockStorageMovementCommand) cmd;
       dn.getStoragePolicySatisfyWorker().processBlockMovingTasks(
-          blkSPSCmd.getTrackID(), blkSPSCmd.getBlockPoolId(),
-          blkSPSCmd.getBlockMovingTasks());
+          blkSPSCmd.getBlockPoolId(), blkSPSCmd.getBlockMovingTasks());
       break;
     case DatanodeProtocol.DNA_DROP_SPS_WORK_COMMAND:
       LOG.info("DatanodeCommand action: DNA_DROP_SPS_WORK_COMMAND");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index f537f49..b7beda4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.client.BlockReportOptions;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -50,7 +51,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -513,8 +514,11 @@ class BPServiceActor implements Runnable {
             SlowDiskReports.create(dn.getDiskMetrics().getDiskOutliersStats()) :
             SlowDiskReports.EMPTY_REPORT;
 
-    BlocksStorageMovementResult[] blksMovementResults =
-        getBlocksMovementResults();
+    // Get the blocks storage move attempt finished blocks
+    List<Block> results = dn.getStoragePolicySatisfyWorker()
+        .getBlocksMovementsStatusHandler().getMoveAttemptFinishedBlocks();
+    BlocksStorageMoveAttemptFinished storageMoveAttemptFinishedBlks =
+        getStorageMoveAttemptFinishedBlocks(results);
 
     HeartbeatResponse response = bpNamenode.sendHeartbeat(bpRegistration,
         reports,
@@ -527,7 +531,7 @@ class BPServiceActor implements Runnable {
         requestBlockReportLease,
         slowPeers,
         slowDisks,
-        blksMovementResults);
+        storageMoveAttemptFinishedBlks);
 
     if (outliersReportDue) {
       // If the report was due and successfully sent, schedule the next one.
@@ -537,20 +541,23 @@ class BPServiceActor implements Runnable {
     // Remove the blocks movement results after successfully transferring
     // to namenode.
     dn.getStoragePolicySatisfyWorker().getBlocksMovementsStatusHandler()
-        .remove(blksMovementResults);
+        .remove(results);
 
     return response;
   }
 
-  private BlocksStorageMovementResult[] getBlocksMovementResults() {
-    List<BlocksStorageMovementResult> trackIdVsMovementStatus = dn
-        .getStoragePolicySatisfyWorker().getBlocksMovementsStatusHandler()
-        .getBlksMovementResults();
-    BlocksStorageMovementResult[] blksMovementResult =
-        new BlocksStorageMovementResult[trackIdVsMovementStatus.size()];
-    trackIdVsMovementStatus.toArray(blksMovementResult);
+  private BlocksStorageMoveAttemptFinished getStorageMoveAttemptFinishedBlocks(
+      List<Block> finishedBlks) {
 
-    return blksMovementResult;
+    if (finishedBlks.isEmpty()) {
+      return null;
+    }
+
+    // Create BlocksStorageMoveAttemptFinished with currently finished
+    // blocks
+    Block[] blockList = new Block[finishedBlks.size()];
+    finishedBlks.toArray(blockList);
+    return new BlocksStorageMoveAttemptFinished(blockList);
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
index f3d2bb6..b3b9fd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -21,14 +21,14 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementResult;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementAttemptFinished;
 import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlocksMovementsStatusHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,12 +41,12 @@ import org.slf4j.LoggerFactory;
 public class BlockStorageMovementTracker implements Runnable {
   private static final Logger LOG = LoggerFactory
       .getLogger(BlockStorageMovementTracker.class);
-  private final CompletionService<BlockMovementResult> moverCompletionService;
+  private final CompletionService<BlockMovementAttemptFinished> moverCompletionService;
   private final BlocksMovementsStatusHandler blksMovementsStatusHandler;
 
-  // Keeps the information - trackID vs its list of blocks
-  private final Map<Long, List<Future<BlockMovementResult>>> moverTaskFutures;
-  private final Map<Long, List<BlockMovementResult>> movementResults;
+  // Keeps the information - block vs its list of future move tasks
+  private final Map<Block, List<Future<BlockMovementAttemptFinished>>> moverTaskFutures;
+  private final Map<Block, List<BlockMovementAttemptFinished>> movementResults;
 
   private volatile boolean running = true;
 
@@ -59,7 +59,7 @@ public class BlockStorageMovementTracker implements Runnable {
    *          blocks movements status handler
    */
   public BlockStorageMovementTracker(
-      CompletionService<BlockMovementResult> moverCompletionService,
+      CompletionService<BlockMovementAttemptFinished> moverCompletionService,
       BlocksMovementsStatusHandler handler) {
     this.moverCompletionService = moverCompletionService;
     this.moverTaskFutures = new HashMap<>();
@@ -82,32 +82,33 @@ public class BlockStorageMovementTracker implements Runnable {
         }
       }
       try {
-        Future<BlockMovementResult> future = moverCompletionService.take();
+        Future<BlockMovementAttemptFinished> future =
+            moverCompletionService.take();
         if (future != null) {
-          BlockMovementResult result = future.get();
+          BlockMovementAttemptFinished result = future.get();
           LOG.debug("Completed block movement. {}", result);
-          long trackId = result.getTrackId();
-          List<Future<BlockMovementResult>> blocksMoving = moverTaskFutures
-              .get(trackId);
+          Block block = result.getBlock();
+          List<Future<BlockMovementAttemptFinished>> blocksMoving =
+              moverTaskFutures.get(block);
           if (blocksMoving == null) {
-            LOG.warn("Future task doesn't exist for trackId " + trackId);
+            LOG.warn("Future task doesn't exist for block : {} ", block);
             continue;
           }
           blocksMoving.remove(future);
 
-          List<BlockMovementResult> resultPerTrackIdList =
-              addMovementResultToTrackIdList(result);
+          List<BlockMovementAttemptFinished> resultPerTrackIdList =
+              addMovementResultToBlockIdList(result);
 
           // Completed all the scheduled blocks movement under this 'trackId'.
-          if (blocksMoving.isEmpty() || moverTaskFutures.get(trackId) == null) {
+          if (blocksMoving.isEmpty() || moverTaskFutures.get(block) == null) {
             synchronized (moverTaskFutures) {
-              moverTaskFutures.remove(trackId);
+              moverTaskFutures.remove(block);
             }
             if (running) {
               // handle completed or inprogress blocks movements per trackId.
               blksMovementsStatusHandler.handle(resultPerTrackIdList);
             }
-            movementResults.remove(trackId);
+            movementResults.remove(block);
           }
         }
       } catch (InterruptedException e) {
@@ -123,38 +124,39 @@ public class BlockStorageMovementTracker implements Runnable {
     }
   }
 
-  private List<BlockMovementResult> addMovementResultToTrackIdList(
-      BlockMovementResult result) {
-    long trackId = result.getTrackId();
-    List<BlockMovementResult> perTrackIdList;
+  private List<BlockMovementAttemptFinished> addMovementResultToBlockIdList(
+      BlockMovementAttemptFinished result) {
+    Block block = result.getBlock();
+    List<BlockMovementAttemptFinished> perBlockIdList;
     synchronized (movementResults) {
-      perTrackIdList = movementResults.get(trackId);
-      if (perTrackIdList == null) {
-        perTrackIdList = new ArrayList<>();
-        movementResults.put(trackId, perTrackIdList);
+      perBlockIdList = movementResults.get(block);
+      if (perBlockIdList == null) {
+        perBlockIdList = new ArrayList<>();
+        movementResults.put(block, perBlockIdList);
       }
-      perTrackIdList.add(result);
+      perBlockIdList.add(result);
     }
-    return perTrackIdList;
+    return perBlockIdList;
   }
 
   /**
    * Add future task to the tracking list to check the completion status of the
    * block movement.
    *
-   * @param trackID
-   *          tracking Id
+   * @param blockID
+   *          block identifier
    * @param futureTask
    *          future task used for moving the respective block
    */
-  void addBlock(long trackID, Future<BlockMovementResult> futureTask) {
+  void addBlock(Block block,
+      Future<BlockMovementAttemptFinished> futureTask) {
     synchronized (moverTaskFutures) {
-      List<Future<BlockMovementResult>> futures = moverTaskFutures
-          .get(Long.valueOf(trackID));
+      List<Future<BlockMovementAttemptFinished>> futures =
+          moverTaskFutures.get(block);
       // null for the first task
       if (futures == null) {
         futures = new ArrayList<>();
-        moverTaskFutures.put(trackID, futures);
+        moverTaskFutures.put(block, futures);
       }
       futures.add(futureTask);
       // Notify waiting tracker thread about the newly added tasks.
@@ -175,16 +177,6 @@ public class BlockStorageMovementTracker implements Runnable {
   }
 
   /**
-   * @return the list of trackIds which are still waiting to complete all the
-   *         scheduled blocks movements.
-   */
-  Set<Long> getInProgressTrackIds() {
-    synchronized (moverTaskFutures) {
-      return moverTaskFutures.keySet();
-    }
-  }
-
-  /**
    * Sets running flag to false and clear the pending movement result queues.
    */
   public void stopTracking() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 4e57805..47318f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
-import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -32,9 +31,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
@@ -62,7 +59,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
@@ -89,14 +85,11 @@ public class StoragePolicySatisfyWorker {
 
   private final int moverThreads;
   private final ExecutorService moveExecutor;
-  private final CompletionService<BlockMovementResult> moverCompletionService;
+  private final CompletionService<BlockMovementAttemptFinished> moverCompletionService;
   private final BlocksMovementsStatusHandler handler;
   private final BlockStorageMovementTracker movementTracker;
   private Daemon movementTrackerThread;
 
-  private long inprogressTrackIdsCheckInterval = 30 * 1000; // 30seconds.
-  private long nextInprogressRecheckTime;
-
   public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
     this.datanode = datanode;
     this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
@@ -111,16 +104,6 @@ public class StoragePolicySatisfyWorker {
     movementTrackerThread = new Daemon(movementTracker);
     movementTrackerThread.setName("BlockStorageMovementTracker");
 
-    // Interval to check that the inprogress trackIds. The time interval is
-    // proportional o the heart beat interval time period.
-    final long heartbeatIntervalSeconds = conf.getTimeDuration(
-        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
-        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
-    inprogressTrackIdsCheckInterval = 5 * heartbeatIntervalSeconds;
-    // update first inprogress recheck time to a future time stamp.
-    nextInprogressRecheckTime = monotonicNow()
-        + inprogressTrackIdsCheckInterval;
-
     // TODO: Needs to manage the number of concurrent moves per DataNode.
   }
 
@@ -186,30 +169,26 @@ public class StoragePolicySatisfyWorker {
    * separate thread. Each task will move the block replica to the target node &
    * wait for the completion.
    *
-   * @param trackID
-   *          unique tracking identifier
-   * @param blockPoolID
-   *          block pool ID
+   * @param blockPoolID block pool identifier
+   *
    * @param blockMovingInfos
    *          list of blocks to be moved
    */
-  public void processBlockMovingTasks(long trackID, String blockPoolID,
-      Collection<BlockMovingInfo> blockMovingInfos) {
+  public void processBlockMovingTasks(final String blockPoolID,
+      final Collection<BlockMovingInfo> blockMovingInfos) {
     LOG.debug("Received BlockMovingTasks {}", blockMovingInfos);
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      assert blkMovingInfo.getSources().length == blkMovingInfo
-          .getTargets().length;
-      for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
-        DatanodeInfo target = blkMovingInfo.getTargets()[i];
-        BlockMovingTask blockMovingTask = new BlockMovingTask(
-            trackID, blockPoolID, blkMovingInfo.getBlock(),
-            blkMovingInfo.getSources()[i], target,
-            blkMovingInfo.getSourceStorageTypes()[i],
-            blkMovingInfo.getTargetStorageTypes()[i]);
-        Future<BlockMovementResult> moveCallable = moverCompletionService
-            .submit(blockMovingTask);
-        movementTracker.addBlock(trackID, moveCallable);
-      }
+      StorageType sourceStorageType = blkMovingInfo.getSourceStorageType();
+      StorageType targetStorageType = blkMovingInfo.getTargetStorageType();
+      assert sourceStorageType != targetStorageType
+          : "Source and Target storage type shouldn't be same!";
+      BlockMovingTask blockMovingTask = new BlockMovingTask(blockPoolID,
+          blkMovingInfo.getBlock(), blkMovingInfo.getSource(),
+          blkMovingInfo.getTarget(), sourceStorageType, targetStorageType);
+      Future<BlockMovementAttemptFinished> moveCallable = moverCompletionService
+          .submit(blockMovingTask);
+      movementTracker.addBlock(blkMovingInfo.getBlock(),
+          moveCallable);
     }
   }
 
@@ -217,8 +196,7 @@ public class StoragePolicySatisfyWorker {
    * This class encapsulates the process of moving the block replica to the
    * given target and wait for the response.
    */
-  private class BlockMovingTask implements Callable<BlockMovementResult> {
-    private final long trackID;
+  private class BlockMovingTask implements Callable<BlockMovementAttemptFinished> {
     private final String blockPoolID;
     private final Block block;
     private final DatanodeInfo source;
@@ -226,10 +204,9 @@ public class StoragePolicySatisfyWorker {
     private final StorageType srcStorageType;
     private final StorageType targetStorageType;
 
-    BlockMovingTask(long trackID, String blockPoolID, Block block,
+    BlockMovingTask(String blockPoolID, Block block,
         DatanodeInfo source, DatanodeInfo target,
         StorageType srcStorageType, StorageType targetStorageType) {
-      this.trackID = trackID;
       this.blockPoolID = blockPoolID;
       this.block = block;
       this.source = source;
@@ -239,23 +216,26 @@ public class StoragePolicySatisfyWorker {
     }
 
     @Override
-    public BlockMovementResult call() {
+    public BlockMovementAttemptFinished call() {
       BlockMovementStatus status = moveBlock();
-      return new BlockMovementResult(trackID, block.getBlockId(), target,
-          status);
+      return new BlockMovementAttemptFinished(block, source, target, status);
     }
 
     private BlockMovementStatus moveBlock() {
       LOG.info("Start moving block:{} from src:{} to destin:{} to satisfy "
-              + "storageType, sourceStoragetype:{} and destinStoragetype:{}",
+          + "storageType, sourceStoragetype:{} and destinStoragetype:{}",
           block, source, target, srcStorageType, targetStorageType);
       Socket sock = null;
       DataOutputStream out = null;
       DataInputStream in = null;
       try {
+        datanode.incrementXmitsInProgress();
+
         ExtendedBlock extendedBlock = new ExtendedBlock(blockPoolID, block);
         DNConf dnConf = datanode.getDnConf();
-        String dnAddr = target.getXferAddr(dnConf.getConnectToDnViaHostname());
+
+        String dnAddr = datanode.getDatanodeId()
+            .getXferAddr(dnConf.getConnectToDnViaHostname());
         sock = datanode.newSocket();
         NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr),
             dnConf.getSocketTimeout());
@@ -297,9 +277,10 @@ public class StoragePolicySatisfyWorker {
         LOG.warn(
             "Failed to move block:{} from src:{} to destin:{} to satisfy "
                 + "storageType:{}",
-            block, source, target, targetStorageType, e);
+                block, source, target, targetStorageType, e);
         return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE;
       } finally {
+        datanode.decrementXmitsInProgress();
         IOUtils.closeStream(out);
         IOUtils.closeStream(in);
         IOUtils.closeSocket(sock);
@@ -357,29 +338,25 @@ public class StoragePolicySatisfyWorker {
   }
 
   /**
-   * This class represents result from a block movement task. This will have the
+   * This class represents status from a block movement task. This will have the
    * information of the task which was successful or failed due to errors.
    */
-  static class BlockMovementResult {
-    private final long trackId;
-    private final long blockId;
+  static class BlockMovementAttemptFinished {
+    private final Block block;
+    private final DatanodeInfo src;
     private final DatanodeInfo target;
     private final BlockMovementStatus status;
 
-    BlockMovementResult(long trackId, long blockId,
+    BlockMovementAttemptFinished(Block block, DatanodeInfo src,
         DatanodeInfo target, BlockMovementStatus status) {
-      this.trackId = trackId;
-      this.blockId = blockId;
+      this.block = block;
+      this.src = src;
       this.target = target;
       this.status = status;
     }
 
-    long getTrackId() {
-      return trackId;
-    }
-
-    long getBlockId() {
-      return blockId;
+    Block getBlock() {
+      return block;
     }
 
     BlockMovementStatus getStatus() {
@@ -388,99 +365,79 @@ public class StoragePolicySatisfyWorker {
 
     @Override
     public String toString() {
-      return new StringBuilder().append("Block movement result(\n  ")
-          .append("track id: ").append(trackId).append(" block id: ")
-          .append(blockId).append(" target node: ").append(target)
+      return new StringBuilder().append("Block movement attempt finished(\n  ")
+          .append(" block : ")
+          .append(block).append(" src node: ").append(src)
+          .append(" target node: ").append(target)
           .append(" movement status: ").append(status).append(")").toString();
     }
   }
 
   /**
    * Blocks movements status handler, which is used to collect details of the
-   * completed or inprogress list of block movements and this status(success or
-   * failure or inprogress) will be send to the namenode via heartbeat.
+   * completed block movements and it will send these attempted finished(with
+   * success or failure) blocks to the namenode via heartbeat.
    */
-  class BlocksMovementsStatusHandler {
-    private final List<BlocksStorageMovementResult> trackIdVsMovementStatus =
+  public static class BlocksMovementsStatusHandler {
+    private final List<Block> blockIdVsMovementStatus =
         new ArrayList<>();
 
     /**
-     * Collect all the block movement results. Later this will be send to
-     * namenode via heart beat.
+     * Collect all the storage movement attempt finished blocks. Later this will
+     * be send to namenode via heart beat.
      *
-     * @param results
-     *          result of all the block movements per trackId
+     * @param moveAttemptFinishedBlks
+     *          set of storage movement attempt finished blocks
      */
-    void handle(List<BlockMovementResult> resultsPerTrackId) {
-      BlocksStorageMovementResult.Status status =
-          BlocksStorageMovementResult.Status.SUCCESS;
-      long trackId = -1;
-      for (BlockMovementResult blockMovementResult : resultsPerTrackId) {
-        trackId = blockMovementResult.getTrackId();
-        if (blockMovementResult.status ==
-            BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE) {
-          status = BlocksStorageMovementResult.Status.FAILURE;
-          // If any of the block movement is failed, then mark as failure so
-          // that namenode can take a decision to retry the blocks associated to
-          // the given trackId.
-          break;
-        }
-      }
+    void handle(List<BlockMovementAttemptFinished> moveAttemptFinishedBlks) {
+      List<Block> blocks = new ArrayList<>();
 
-      // Adding to the tracking results list. Later this will be send to
+      for (BlockMovementAttemptFinished item : moveAttemptFinishedBlks) {
+        blocks.add(item.getBlock());
+      }
+      // Adding to the tracking report list. Later this will be send to
       // namenode via datanode heartbeat.
-      synchronized (trackIdVsMovementStatus) {
-        trackIdVsMovementStatus.add(
-            new BlocksStorageMovementResult(trackId, status));
+      synchronized (blockIdVsMovementStatus) {
+        blockIdVsMovementStatus.addAll(blocks);
       }
     }
 
     /**
-     * @return unmodifiable list of blocks storage movement results.
+     * @return unmodifiable list of storage movement attempt finished blocks.
      */
-    List<BlocksStorageMovementResult> getBlksMovementResults() {
-      List<BlocksStorageMovementResult> movementResults = new ArrayList<>();
-      // 1. Adding all the completed trackids.
-      synchronized (trackIdVsMovementStatus) {
-        if (trackIdVsMovementStatus.size() > 0) {
-          movementResults = Collections
-              .unmodifiableList(trackIdVsMovementStatus);
+    List<Block> getMoveAttemptFinishedBlocks() {
+      List<Block> moveAttemptFinishedBlks = new ArrayList<>();
+      // 1. Adding all the completed block ids.
+      synchronized (blockIdVsMovementStatus) {
+        if (blockIdVsMovementStatus.size() > 0) {
+          moveAttemptFinishedBlks = Collections
+              .unmodifiableList(blockIdVsMovementStatus);
         }
       }
-      // 2. Adding the in progress track ids after those which are completed.
-      Set<Long> inProgressTrackIds = getInProgressTrackIds();
-      for (Long trackId : inProgressTrackIds) {
-        movementResults.add(new BlocksStorageMovementResult(trackId,
-            BlocksStorageMovementResult.Status.IN_PROGRESS));
-      }
-      return movementResults;
+      return moveAttemptFinishedBlks;
     }
 
     /**
-     * Remove the blocks storage movement results.
+     * Remove the storage movement attempt finished blocks from the tracking
+     * list.
      *
-     * @param results
-     *          set of blocks storage movement results
+     * @param moveAttemptFinishedBlks
+     *          set of storage movement attempt finished blocks
      */
-    void remove(BlocksStorageMovementResult[] results) {
-      if (results != null) {
-        synchronized (trackIdVsMovementStatus) {
-          for (BlocksStorageMovementResult blocksMovementResult : results) {
-            trackIdVsMovementStatus.remove(blocksMovementResult);
-          }
-        }
+    void remove(List<Block> moveAttemptFinishedBlks) {
+      if (moveAttemptFinishedBlks != null) {
+        blockIdVsMovementStatus.removeAll(moveAttemptFinishedBlks);
       }
     }
 
     /**
-     * Clear the trackID vs movement status tracking map.
+     * Clear the blockID vs movement status tracking map.
      */
     void removeAll() {
-      synchronized (trackIdVsMovementStatus) {
-        trackIdVsMovementStatus.clear();
+      synchronized (blockIdVsMovementStatus) {
+        blockIdVsMovementStatus.clear();
       }
     }
-
   }
 
   @VisibleForTesting
@@ -498,23 +455,4 @@ public class StoragePolicySatisfyWorker {
     movementTracker.removeAll();
     handler.removeAll();
   }
-
-  /**
-   * Gets list of trackids which are inprogress. Will do collection periodically
-   * on 'dfs.datanode.storage.policy.satisfier.worker.inprogress.recheck.time.
-   * millis' interval.
-   *
-   * @return collection of trackids which are inprogress
-   */
-  private Set<Long> getInProgressTrackIds() {
-    Set<Long> trackIds = new HashSet<>();
-    long now = monotonicNow();
-    if (nextInprogressRecheckTime >= now) {
-      trackIds = movementTracker.getInProgressTrackIds();
-
-      // schedule next re-check interval
-      nextInprogressRecheckTime = now + inprogressTrackIdsCheckInterval;
-    }
-    return trackIds;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 549819f..cc5b63a 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
@@ -22,15 +22,12 @@ import static org.apache.hadoop.util.Time.monotonicNow;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.AttemptedItemInfo;
 import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult.Status;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,14 +35,12 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- * A monitor class for checking whether block storage movements finished or not.
- * If block storage movement results from datanode indicates about the movement
- * success, then it will just remove the entries from tracking. If it reports
- * failure, then it will add back to needed block storage movements list. If it
- * reports in_progress, that means the blocks movement is in progress and the
- * coordinator is still tracking the movement. If no DN reports about movement
- * for longer time, then such items will be retries automatically after timeout.
- * The default timeout would be 30mins.
+ * A monitor class for checking whether block storage movements attempt
+ * completed or not. If this receives block storage movement attempt
+ * status(either success or failure) from DN then it will just remove the
+ * entries from tracking. If there is no DN reports about movement attempt
+ * finished for a longer time period, then such items will retries automatically
+ * after timeout. The default timeout would be 5 minutes.
  */
 public class BlockStorageMovementAttemptedItems {
   private static final Logger LOG =
@@ -55,37 +50,34 @@ public class BlockStorageMovementAttemptedItems {
    * A map holds the items which are already taken for blocks movements
    * processing and sent to DNs.
    */
-  private final Map<Long, AttemptedItemInfo> storageMovementAttemptedItems;
-  private final List<BlocksStorageMovementResult> storageMovementAttemptedResults;
+  private final List<AttemptedItemInfo> storageMovementAttemptedItems;
+  private final List<Block> movementFinishedBlocks;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
-  private final StoragePolicySatisfier sps;
   //
-  // It might take anywhere between 20 to 60 minutes before
+  // It might take anywhere between 5 to 10 minutes before
   // a request is timed out.
   //
-  private long selfRetryTimeout = 20 * 60 * 1000;
+  private long selfRetryTimeout = 5 * 60 * 1000;
 
   //
-  // It might take anywhere between 5 to 10 minutes before
+  // It might take anywhere between 1 to 2 minutes before
   // a request is timed out.
   //
-  private long minCheckTimeout = 5 * 60 * 1000; // minimum value
+  private long minCheckTimeout = 1 * 60 * 1000; // minimum value
   private BlockStorageMovementNeeded blockStorageMovementNeeded;
 
   public BlockStorageMovementAttemptedItems(long recheckTimeout,
       long selfRetryTimeout,
-      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles,
-      StoragePolicySatisfier sps) {
+      BlockStorageMovementNeeded unsatisfiedStorageMovementFiles) {
     if (recheckTimeout > 0) {
       this.minCheckTimeout = Math.min(minCheckTimeout, recheckTimeout);
     }
 
     this.selfRetryTimeout = selfRetryTimeout;
     this.blockStorageMovementNeeded = unsatisfiedStorageMovementFiles;
-    storageMovementAttemptedItems = new HashMap<>();
-    storageMovementAttemptedResults = new ArrayList<>();
-    this.sps = sps;
+    storageMovementAttemptedItems = new ArrayList<>();
+    movementFinishedBlocks = new ArrayList<>();
   }
 
   /**
@@ -94,33 +86,26 @@ public class BlockStorageMovementAttemptedItems {
    *
    * @param itemInfo
    *          - tracking info
-   * @param allBlockLocsAttemptedToSatisfy
-   *          - failed to find matching target nodes to satisfy storage type
-   *          for all the block locations of the given blockCollectionID
    */
-  public void add(ItemInfo itemInfo, boolean allBlockLocsAttemptedToSatisfy) {
+  public void add(AttemptedItemInfo itemInfo) {
     synchronized (storageMovementAttemptedItems) {
-      AttemptedItemInfo attemptedItemInfo = new AttemptedItemInfo(
-          itemInfo.getStartId(), itemInfo.getTrackId(), monotonicNow(),
-          allBlockLocsAttemptedToSatisfy);
-      storageMovementAttemptedItems.put(itemInfo.getTrackId(),
-          attemptedItemInfo);
+      storageMovementAttemptedItems.add(itemInfo);
     }
   }
 
   /**
-   * Add the trackIDBlocksStorageMovementResults to
-   * storageMovementAttemptedResults.
+   * Add the storage movement attempt finished blocks to
+   * storageMovementFinishedBlocks.
    *
-   * @param blksMovementResults
+   * @param moveAttemptFinishedBlks
+   *          storage movement attempt finished blocks
    */
-  public void addResults(BlocksStorageMovementResult[] blksMovementResults) {
-    if (blksMovementResults.length == 0) {
+  public void addReportedMovedBlocks(Block[] moveAttemptFinishedBlks) {
+    if (moveAttemptFinishedBlks.length == 0) {
       return;
     }
-    synchronized (storageMovementAttemptedResults) {
-      storageMovementAttemptedResults
-          .addAll(Arrays.asList(blksMovementResults));
+    synchronized (movementFinishedBlocks) {
+      movementFinishedBlocks.addAll(Arrays.asList(moveAttemptFinishedBlks));
     }
   }
 
@@ -129,8 +114,8 @@ public class BlockStorageMovementAttemptedItems {
    */
   public synchronized void start() {
     monitorRunning = true;
-    timerThread = new Daemon(new BlocksStorageMovementAttemptResultMonitor());
-    timerThread.setName("BlocksStorageMovementAttemptResultMonitor");
+    timerThread = new Daemon(new BlocksStorageMovementAttemptMonitor());
+    timerThread.setName("BlocksStorageMovementAttemptMonitor");
     timerThread.start();
   }
 
@@ -163,82 +148,22 @@ public class BlockStorageMovementAttemptedItems {
   }
 
   /**
-   * This class contains information of an attempted trackID. Information such
-   * as, (a)last attempted or reported time stamp, (b)whether all the blocks in
-   * the trackID were attempted and blocks movement has been scheduled to
-   * satisfy storage policy. This is used by
-   * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
-   */
-  private final static class AttemptedItemInfo extends ItemInfo {
-    private long lastAttemptedOrReportedTime;
-    private final boolean allBlockLocsAttemptedToSatisfy;
-
-    /**
-     * AttemptedItemInfo constructor.
-     *
-     * @param rootId
-     *          rootId for trackId
-     * @param trackId
-     *          trackId for file.
-     * @param lastAttemptedOrReportedTime
-     *          last attempted or reported time
-     * @param allBlockLocsAttemptedToSatisfy
-     *          whether all the blocks in the trackID were attempted and blocks
-     *          movement has been scheduled to satisfy storage policy
-     */
-    private AttemptedItemInfo(long rootId, long trackId,
-        long lastAttemptedOrReportedTime,
-        boolean allBlockLocsAttemptedToSatisfy) {
-      super(rootId, trackId);
-      this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
-      this.allBlockLocsAttemptedToSatisfy = allBlockLocsAttemptedToSatisfy;
-    }
-
-    /**
-     * @return last attempted or reported time stamp.
-     */
-    private long getLastAttemptedOrReportedTime() {
-      return lastAttemptedOrReportedTime;
-    }
-
-    /**
-     * @return true/false. True value represents that, all the block locations
-     *         under the trackID has found matching target nodes to satisfy
-     *         storage policy. False value represents that, trackID needed
-     *         retries to satisfy the storage policy for some of the block
-     *         locations.
-     */
-    private boolean isAllBlockLocsAttemptedToSatisfy() {
-      return allBlockLocsAttemptedToSatisfy;
-    }
-
-    /**
-     * Update lastAttemptedOrReportedTime, so that the expiration time will be
-     * postponed to future.
-     */
-    private void touchLastReportedTimeStamp() {
-      this.lastAttemptedOrReportedTime = monotonicNow();
-    }
-
-  }
-
-  /**
-   * A monitor class for checking block storage movement result and long waiting
-   * items periodically.
+   * A monitor class for checking block storage movement attempt status and long
+   * waiting items periodically.
    */
-  private class BlocksStorageMovementAttemptResultMonitor implements Runnable {
+  private class BlocksStorageMovementAttemptMonitor implements Runnable {
     @Override
     public void run() {
       while (monitorRunning) {
         try {
-          blockStorageMovementResultCheck();
+          blockStorageMovementReportedItemsCheck();
           blocksStorageMovementUnReportedItemsCheck();
           Thread.sleep(minCheckTimeout);
         } catch (InterruptedException ie) {
-          LOG.info("BlocksStorageMovementAttemptResultMonitor thread "
+          LOG.info("BlocksStorageMovementAttemptMonitor thread "
               + "is interrupted.", ie);
         } catch (IOException ie) {
-          LOG.warn("BlocksStorageMovementAttemptResultMonitor thread "
+          LOG.warn("BlocksStorageMovementAttemptMonitor thread "
               + "received exception and exiting.", ie);
         }
       }
@@ -248,29 +173,21 @@ public class BlockStorageMovementAttemptedItems {
   @VisibleForTesting
   void blocksStorageMovementUnReportedItemsCheck() {
     synchronized (storageMovementAttemptedItems) {
-      Iterator<Entry<Long, AttemptedItemInfo>> iter =
-          storageMovementAttemptedItems.entrySet().iterator();
+      Iterator<AttemptedItemInfo> iter = storageMovementAttemptedItems
+          .iterator();
       long now = monotonicNow();
       while (iter.hasNext()) {
-        Entry<Long, AttemptedItemInfo> entry = iter.next();
-        AttemptedItemInfo itemInfo = entry.getValue();
+        AttemptedItemInfo itemInfo = iter.next();
         if (now > itemInfo.getLastAttemptedOrReportedTime()
             + selfRetryTimeout) {
-          Long blockCollectionID = entry.getKey();
-          synchronized (storageMovementAttemptedResults) {
-            if (!isExistInResult(blockCollectionID)) {
-              ItemInfo candidate = new ItemInfo(
-                  itemInfo.getStartId(), blockCollectionID);
-              blockStorageMovementNeeded.add(candidate);
-              iter.remove();
-              LOG.info("TrackID: {} becomes timed out and moved to needed "
-                  + "retries queue for next iteration.", blockCollectionID);
-            } else {
-              LOG.info("Blocks storage movement results for the"
-                  + " tracking id : " + blockCollectionID
-                  + " is reported from one of the co-ordinating datanode."
-                  + " So, the result will be processed soon.");
-            }
+          Long blockCollectionID = itemInfo.getTrackId();
+          synchronized (movementFinishedBlocks) {
+            ItemInfo candidate = new ItemInfo(itemInfo.getStartId(),
+                blockCollectionID);
+            blockStorageMovementNeeded.add(candidate);
+            iter.remove();
+            LOG.info("TrackID: {} becomes timed out and moved to needed "
+                + "retries queue for next iteration.", blockCollectionID);
           }
         }
       }
@@ -278,118 +195,38 @@ public class BlockStorageMovementAttemptedItems {
     }
   }
 
-  private boolean isExistInResult(Long blockCollectionID) {
-    Iterator<BlocksStorageMovementResult> iter = storageMovementAttemptedResults
-        .iterator();
-    while (iter.hasNext()) {
-      BlocksStorageMovementResult storageMovementAttemptedResult = iter.next();
-      if (storageMovementAttemptedResult.getTrackId() == blockCollectionID) {
-        return true;
-      }
-    }
-    return false;
-  }
-
   @VisibleForTesting
-  void blockStorageMovementResultCheck() throws IOException {
-    synchronized (storageMovementAttemptedResults) {
-      Iterator<BlocksStorageMovementResult> resultsIter =
-          storageMovementAttemptedResults.iterator();
-      while (resultsIter.hasNext()) {
-        boolean isInprogress = false;
-        // TrackID need to be retried in the following cases:
-        // 1) All or few scheduled block(s) movement has been failed.
-        // 2) All the scheduled block(s) movement has been succeeded but there
-        // are unscheduled block(s) movement in this trackID. Say, some of
-        // the blocks in the trackID couldn't finding any matching target node
-        // for scheduling block movement in previous SPS iteration.
-        BlocksStorageMovementResult storageMovementAttemptedResult = resultsIter
-            .next();
+  void blockStorageMovementReportedItemsCheck() throws IOException {
+    synchronized (movementFinishedBlocks) {
+      Iterator<Block> finishedBlksIter = movementFinishedBlocks.iterator();
+      while (finishedBlksIter.hasNext()) {
+        Block blk = finishedBlksIter.next();
         synchronized (storageMovementAttemptedItems) {
-          Status status = storageMovementAttemptedResult.getStatus();
-          long trackId = storageMovementAttemptedResult.getTrackId();
-          AttemptedItemInfo attemptedItemInfo = storageMovementAttemptedItems
-              .get(trackId);
-          // itemInfo is null means no root for trackId, using trackId only as
-          // root and handling it in
-          // blockStorageMovementNeeded#removeIteamTrackInfo() for cleaning
-          // the xAttr
-          ItemInfo itemInfo = new ItemInfo((attemptedItemInfo != null)
-              ? attemptedItemInfo.getStartId() : trackId, trackId);
-          switch (status) {
-          case FAILURE:
-            if (attemptedItemInfo != null) {
-              blockStorageMovementNeeded.add(itemInfo);
-              LOG.warn("Blocks storage movement results for the tracking id:"
-                  + "{} is reported from co-ordinating datanode, but result"
-                  + " status is FAILURE. So, added for retry", trackId);
-            } else {
-              LOG.info("Blocks storage movement is FAILURE for the track"
-                  + " id {}. But the trackID doesn't exists in"
-                  + " storageMovementAttemptedItems list.", trackId);
-              blockStorageMovementNeeded
-                  .removeItemTrackInfo(itemInfo);
-            }
-            break;
-          case SUCCESS:
-            // ItemInfo could be null. One case is, before the blocks movements
-            // result arrives the attempted trackID became timed out and then
-            // removed the trackID from the storageMovementAttemptedItems list.
-            // TODO: Need to ensure that trackID is added to the
-            // 'blockStorageMovementNeeded' queue for retries to handle the
-            // following condition. If all the block locations under the trackID
-            // are attempted and failed to find matching target nodes to satisfy
-            // storage policy in previous SPS iteration.
-            String msg = "Blocks storage movement is SUCCESS for the track id: "
-                + trackId + " reported from co-ordinating datanode.";
-            if (attemptedItemInfo != null) {
-              if (!attemptedItemInfo.isAllBlockLocsAttemptedToSatisfy()) {
-                blockStorageMovementNeeded
-                    .add(new ItemInfo(attemptedItemInfo.getStartId(), trackId));
-                LOG.warn("{} But adding trackID back to retry queue as some of"
-                    + " the blocks couldn't find matching target nodes in"
-                    + " previous SPS iteration.", msg);
-              } else {
-                LOG.info(msg);
-                blockStorageMovementNeeded
-                    .removeItemTrackInfo(itemInfo);
-              }
-            } else {
-              LOG.info("{} But the trackID doesn't exists in "
-                  + "storageMovementAttemptedItems list", msg);
+          Iterator<AttemptedItemInfo> iterator = storageMovementAttemptedItems
+              .iterator();
+          while (iterator.hasNext()) {
+            AttemptedItemInfo attemptedItemInfo = iterator.next();
+            attemptedItemInfo.getBlocks().remove(blk);
+            if (attemptedItemInfo.getBlocks().isEmpty()) {
+              // TODO: try add this at front of the Queue, so that this element
+              // gets the chance first and can be cleaned from queue quickly as
+              // all movements already done.
               blockStorageMovementNeeded
-              .removeItemTrackInfo(itemInfo);
-            }
-            break;
-          case IN_PROGRESS:
-            isInprogress = true;
-            attemptedItemInfo = storageMovementAttemptedItems
-                .get(storageMovementAttemptedResult.getTrackId());
-            if(attemptedItemInfo != null){
-              // update the attempted expiration time to next cycle.
-              attemptedItemInfo.touchLastReportedTimeStamp();
+                  .add(new ItemInfo(attemptedItemInfo.getStartId(),
+                      attemptedItemInfo.getTrackId()));
+              iterator.remove();
             }
-            break;
-          default:
-            LOG.error("Unknown status: {}", status);
-            break;
-          }
-          // Remove trackID from the attempted list if the attempt has been
-          // completed(success or failure), if any.
-          if (!isInprogress) {
-            storageMovementAttemptedItems
-                .remove(storageMovementAttemptedResult.getTrackId());
           }
         }
-        // Remove trackID from results as processed above.
-        resultsIter.remove();
+        // Remove attempted blocks from movementFinishedBlocks list.
+        finishedBlksIter.remove();
       }
     }
   }
 
   @VisibleForTesting
-  public int resultsCount() {
-    return storageMovementAttemptedResults.size();
+  public int getMovementFinishedBlocksCount() {
+    return movementFinishedBlocks.size();
   }
 
   @VisibleForTesting
@@ -398,7 +235,7 @@ public class BlockStorageMovementAttemptedItems {
   }
 
   public void clearQueues() {
-    storageMovementAttemptedResults.clear();
+    movementFinishedBlocks.clear();
     storageMovementAttemptedItems.clear();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java
deleted file mode 100644
index a790c13..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.util.List;
-
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-
-/**
- * This class represents a batch of blocks under one trackId which needs to move
- * its storage locations to satisfy the storage policy.
- */
-public class BlockStorageMovementInfosBatch {
-  private long trackID;
-  private List<BlockMovingInfo> blockMovingInfos;
-
-  /**
-   * Constructor to create the block storage movement infos batch.
-   *
-   * @param trackID
-   *          - unique identifier which will be used for tracking the given set
-   *          of blocks movement.
-   * @param blockMovingInfos
-   *          - list of block to storage infos.
-   */
-  public BlockStorageMovementInfosBatch(long trackID,
-      List<BlockMovingInfo> blockMovingInfos) {
-    this.trackID = trackID;
-    this.blockMovingInfos = blockMovingInfos;
-  }
-
-  public long getTrackID() {
-    return trackID;
-  }
-
-  public List<BlockMovingInfo> getBlockMovingInfo() {
-    return blockMovingInfos;
-  }
-
-  @Override
-  public String toString() {
-    return new StringBuilder().append("BlockStorageMovementInfosBatch(\n  ")
-        .append("TrackID: ").append(trackID).append("  BlockMovingInfos: ")
-        .append(blockMovingInfos).append(")").toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 1e09a4d..7eb156c 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
@@ -265,7 +265,7 @@ import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
 import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics;
 import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -3822,7 +3822,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
       @Nonnull SlowDiskReports slowDisks,
-      BlocksStorageMovementResult[] blksMovementResults) throws IOException {
+      BlocksStorageMoveAttemptFinished blksMovementsFinished)
+          throws IOException {
     readLock();
     try {
       //get datanode commands
@@ -3843,11 +3844,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         if (!sps.isRunning()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug(
-                "Storage policy satisfier is not running. So, ignoring block "
-                    + "storage movement results sent by co-ordinator datanode");
+                "Storage policy satisfier is not running. So, ignoring storage"
+                    + "  movement attempt finished block info sent by DN");
           }
         } else {
-          sps.handleBlocksStorageMovementResults(blksMovementResults);
+          sps.handleStorageMovementAttemptFinishedBlks(blksMovementsFinished);
         }
       }
 


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


[31/50] [abbrv] hadoop git commit: HDFS-11726. [SPS]: StoragePolicySatisfier should not select same storage type as source and destination in same datanode. Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-11726. [SPS]: StoragePolicySatisfier should not select same storage type as source and destination in same datanode. Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 121dc7a928dc8825a9b44b8a71b2e4736157fbd6
Parents: 8feeadb
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Jun 9 14:03:13 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:02 2017 +0530

----------------------------------------------------------------------
 .../server/namenode/StoragePolicySatisfier.java | 23 ++++++----
 .../namenode/TestStoragePolicySatisfier.java    | 44 ++++++++++++++++++++
 2 files changed, 58 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/121dc7a9/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 9e2a4a0..1b2afa3 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
@@ -501,15 +501,20 @@ public class StoragePolicySatisfier implements Runnable {
     // avoid choosing a target which already has this block.
     for (int i = 0; i < sourceWithStorageList.size(); i++) {
       StorageTypeNodePair existingTypeNodePair = sourceWithStorageList.get(i);
-      StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(blockInfo,
-          existingTypeNodePair.dn, expected);
-      if (chosenTarget != null) {
-        sourceNodes.add(existingTypeNodePair.dn);
-        sourceStorageTypes.add(existingTypeNodePair.storageType);
-        targetNodes.add(chosenTarget.dn);
-        targetStorageTypes.add(chosenTarget.storageType);
-        expected.remove(chosenTarget.storageType);
-        // TODO: We can increment scheduled block count for this node?
+
+      // Check whether the block replica is already placed in the expected
+      // storage type in this source datanode.
+      if (!expected.contains(existingTypeNodePair.storageType)) {
+        StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(
+            blockInfo, existingTypeNodePair.dn, expected);
+        if (chosenTarget != null) {
+          sourceNodes.add(existingTypeNodePair.dn);
+          sourceStorageTypes.add(existingTypeNodePair.storageType);
+          targetNodes.add(chosenTarget.dn);
+          targetStorageTypes.add(chosenTarget.storageType);
+          expected.remove(chosenTarget.storageType);
+          // TODO: We can increment scheduled block count for this node?
+        }
       }
       // To avoid choosing this excludeNodes as targets later
       excludeNodes.add(existingTypeNodePair.dn);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/121dc7a9/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 8e08a1e..f1a4169 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
@@ -764,6 +764,50 @@ public class TestStoragePolicySatisfier {
   }
 
   /**
+   * If replica with expected storage type already exist in source DN then that
+   * DN should be skipped.
+   */
+  @Test(timeout = 300000)
+  public void testSPSWhenReplicaWithExpectedStorageAlreadyAvailableInSource()
+      throws Exception {
+    StorageType[][] diskTypes = new StorageType[][] {
+        {StorageType.DISK, StorageType.ARCHIVE},
+        {StorageType.DISK, StorageType.ARCHIVE},
+        {StorageType.DISK, StorageType.ARCHIVE}};
+
+    try {
+      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+          storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      // 1. Write two replica on disk
+      DFSTestUtil.createFile(dfs, new Path(file), DEFAULT_BLOCK_SIZE,
+          (short) 2, 0);
+      // 2. Change policy to COLD, so third replica will be written to ARCHIVE.
+      dfs.setStoragePolicy(new Path(file), "COLD");
+
+      // 3.Change replication factor to 3.
+      dfs.setReplication(new Path(file), (short) 3);
+
+      DFSTestUtil
+          .waitExpectedStorageType(file, StorageType.DISK, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
+          dfs);
+
+      // 4. Change policy to HOT, so we can move the all block to DISK.
+      dfs.setStoragePolicy(new Path(file), "HOT");
+
+      // 4. Satisfy the policy.
+      dfs.satisfyStoragePolicy(new Path(file));
+
+      // 5. Block should move successfully .
+      DFSTestUtil
+          .waitExpectedStorageType(file, StorageType.DISK, 3, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
+  /**
    * Tests that movements should not be assigned when there is no space in
    * target DN.
    */


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


[25/50] [abbrv] hadoop git commit: HDFS-11338: [SPS]: Fix timeout issue in unit tests caused by longger NN down time. Contributed by Wei Zhou and Rakesh R

Posted by ra...@apache.org.
HDFS-11338: [SPS]: Fix timeout issue in unit tests caused by longger NN down time. Contributed by Wei Zhou and Rakesh R


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

Branch: refs/heads/HDFS-10285
Commit: af158035179722d918e0b5cbe3ac9a60c102dab4
Parents: cbbe71f
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue Apr 11 14:25:01 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:22 2017 +0530

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 13 +++++--
 .../BlockStorageMovementAttemptedItems.java     | 25 +++++++++----
 .../hdfs/server/namenode/FSNamesystem.java      |  2 +-
 .../server/namenode/StoragePolicySatisfier.java | 38 ++++++++++++++------
 .../TestBlockStorageMovementAttemptedItems.java |  3 +-
 5 files changed, 60 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/af158035/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 def3059..d1bb570 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
@@ -728,7 +728,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   public void close() {
     if (sps != null) {
-      sps.stop(false);
+      sps.deactivate(false);
     }
     bmSafeMode.close();
     try {
@@ -743,6 +743,7 @@ public class BlockManager implements BlockStatsMXBean {
     datanodeManager.close();
     pendingReconstruction.stop();
     blocksMap.close();
+    stopSPSGracefully();
   }
 
   /** @return the datanodeManager */
@@ -4989,10 +4990,18 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is already stopped.");
       return;
     }
-    sps.stop(true);
+    sps.deactivate(true);
   }
 
   /**
+   * Timed wait to stop storage policy satisfier daemon threads.
+   */
+  public void stopSPSGracefully() {
+    if (sps != null) {
+      sps.stopGracefully();
+    }
+  }
+  /**
    * @return True if storage policy satisfier running.
    */
   public boolean isStoragePolicySatisfierRunning() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af158035/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 f15db73..26b98d8 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
@@ -130,21 +130,34 @@ public class BlockStorageMovementAttemptedItems {
   }
 
   /**
-   * Stops the monitor thread.
+   * Sets running flag to false. Also, this will interrupt monitor thread and
+   * clear all the queued up tasks.
    */
-  public synchronized void stop() {
+  public synchronized void deactivate() {
     monitorRunning = false;
     if (timerThread != null) {
       timerThread.interrupt();
-      try {
-        timerThread.join(3000);
-      } catch (InterruptedException ie) {
-      }
     }
     this.clearQueues();
   }
 
   /**
+   * Timed wait to stop monitor thread.
+   */
+  synchronized void stopGracefully() {
+    if (timerThread == null) {
+      return;
+    }
+    if (monitorRunning) {
+      deactivate();
+    }
+    try {
+      timerThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  /**
    * This class contains information of an attempted trackID. Information such
    * as, (a)last attempted time stamp, (b)whether all the blocks in the trackID
    * were attempted and blocks movement has been scheduled to satisfy storage

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af158035/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 0be96f3..55f3e84 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
@@ -1318,7 +1318,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (blockManager != null) {
         blockManager.deactivateSPS();
       }
-
       stopSecretManager();
       leaseManager.stopMonitor();
       if (nnrmthread != null) {
@@ -1357,6 +1356,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // Don't want to keep replication queues when not in Active.
         blockManager.clearQueues();
         blockManager.setInitializedReplQueues(false);
+        blockManager.stopSPSGracefully();
       }
     } finally {
       writeUnlock("stopActiveServices");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af158035/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 337d5b5..8cf9920 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
@@ -115,22 +115,21 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   /**
-   * Stop storage policy satisfier demon thread.
+   * Deactivates storage policy satisfier by stopping its services.
    *
-   * @param reconfigStop
+   * @param reconfig
+   *          true represents deactivating SPS service as requested by admin,
+   *          false otherwise
    */
-  public synchronized void stop(boolean reconfigStop) {
+  public synchronized void deactivate(boolean reconfig) {
     isRunning = false;
     if (storagePolicySatisfierThread == null) {
       return;
     }
+
     storagePolicySatisfierThread.interrupt();
-    try {
-      storagePolicySatisfierThread.join(3000);
-    } catch (InterruptedException ie) {
-    }
-    this.storageMovementsMonitor.stop();
-    if (reconfigStop) {
+    this.storageMovementsMonitor.deactivate();
+    if (reconfig) {
       LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
           + "deactivate it.");
       this.clearQueuesWithNotification();
@@ -141,6 +140,23 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   /**
+   * Timed wait to stop storage policy satisfier daemon threads.
+   */
+  public synchronized void stopGracefully() {
+    if (isRunning) {
+      deactivate(true);
+    }
+    this.storageMovementsMonitor.stopGracefully();
+    if (storagePolicySatisfierThread == null) {
+      return;
+    }
+    try {
+      storagePolicySatisfierThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  /**
    * Check whether StoragePolicySatisfier is running.
    * @return true if running
    */
@@ -162,7 +178,7 @@ public class StoragePolicySatisfier implements Runnable {
       if (!isRunning) {
         // Stopping monitor thread and clearing queues as well
         this.clearQueues();
-        this.storageMovementsMonitor.stop();
+        this.storageMovementsMonitor.stopGracefully();
         LOG.error(
             "Stopping StoragePolicySatisfier thread " + "as Mover ID file "
                 + HdfsServerConstants.MOVER_ID_PATH.toString()
@@ -194,7 +210,7 @@ public class StoragePolicySatisfier implements Runnable {
           isRunning = false;
           // Stopping monitor thread and clearing queues as well
           this.clearQueues();
-          this.storageMovementsMonitor.stop();
+          this.storageMovementsMonitor.stopGracefully();
         }
         if (!namesystem.isRunning()) {
           LOG.info("Stopping StoragePolicySatisfier.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af158035/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 95142d3..8c7d982 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,8 @@ public class TestBlockStorageMovementAttemptedItems {
   @After
   public void teardown() {
     if (bsmAttemptedItems != null) {
-      bsmAttemptedItems.stop();
+      bsmAttemptedItems.deactivate();
+      bsmAttemptedItems.stopGracefully();
     }
   }
 


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


[02/50] [abbrv] hadoop git commit: YARN-7608. Incorrect sTarget column causing DataTable warning on RM application and scheduler web page. Contributed by Gergely Novák.

Posted by ra...@apache.org.
YARN-7608. Incorrect sTarget column causing DataTable warning on RM application and scheduler web page. Contributed by Gergely Novák.


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

Branch: refs/heads/HDFS-10285
Commit: a2edc4cbf5c1f7bc38315c52391362fbbc48fab1
Parents: 670e8d4
Author: Weiwei Yang <ww...@apache.org>
Authored: Mon Dec 11 10:31:46 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Mon Dec 11 10:31:46 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2edc4cb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
index 3a26ae5..29f1e63 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
@@ -55,9 +55,9 @@ public class WebPageUtils {
       .append(", 'mRender': renderHadoopDate }")
       .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets':");
     if (isFairSchedulerPage) {
-      sb.append("[13]");
-    } else if (isResourceManager) {
       sb.append("[15]");
+    } else if (isResourceManager) {
+      sb.append("[17]");
     } else {
       sb.append("[9]");
     }


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


[22/50] [abbrv] hadoop git commit: HDFS-11243. [SPS]: Add a protocol command from NN to DN for dropping the SPS work and queues. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11243. [SPS]: Add a protocol command from NN to DN for dropping the SPS work and queues. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: 18d9bc520f759a8c71fe98f65d97587d96b3c52b
Parents: e6e6a59
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Tue Jan 31 23:44:01 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:07 2017 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 12 ++++
 .../server/blockmanagement/BlockManager.java    | 13 ++---
 .../blockmanagement/DatanodeDescriptor.java     | 18 ++++++
 .../server/blockmanagement/DatanodeManager.java | 19 +++++++
 .../hdfs/server/datanode/BPOfferService.java    |  4 ++
 .../datanode/BlockStorageMovementTracker.java   | 12 ++++
 .../datanode/StoragePolicySatisfyWorker.java    | 22 +++++++-
 .../server/namenode/StoragePolicySatisfier.java | 25 +++++++--
 .../hdfs/server/protocol/DatanodeProtocol.java  |  2 +
 .../server/protocol/DropSPSWorkCommand.java     | 36 ++++++++++++
 .../src/main/proto/DatanodeProtocol.proto       |  9 +++
 .../TestStoragePolicySatisfyWorker.java         | 59 ++++++++++++++++++++
 12 files changed, 216 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 156c9c2..3b38077 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdComma
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DropSPSWorkCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
@@ -108,6 +109,7 @@ import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DropSPSWorkCommand;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
@@ -139,6 +141,10 @@ public class PBHelper {
   private static final RegisterCommandProto REG_CMD_PROTO = 
       RegisterCommandProto.newBuilder().build();
   private static final RegisterCommand REG_CMD = new RegisterCommand();
+  private static final DropSPSWorkCommandProto DROP_SPS_WORK_CMD_PROTO =
+      DropSPSWorkCommandProto.newBuilder().build();
+  private static final DropSPSWorkCommand DROP_SPS_WORK_CMD =
+      new DropSPSWorkCommand();
 
   private PBHelper() {
     /** Hidden constructor */
@@ -474,6 +480,8 @@ public class PBHelper {
       return PBHelper.convert(proto.getBlkECReconstructionCmd());
     case BlockStorageMovementCommand:
       return PBHelper.convert(proto.getBlkStorageMovementCmd());
+    case DropSPSWorkCommand:
+      return DROP_SPS_WORK_CMD;
     default:
       return null;
     }
@@ -613,6 +621,10 @@ public class PBHelper {
           .setBlkStorageMovementCmd(
               convert((BlockStorageMovementCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_DROP_SPS_WORK_COMMAND:
+      builder.setCmdType(DatanodeCommandProto.Type.DropSPSWorkCommand)
+          .setDropSPSWorkCmd(DROP_SPS_WORK_CMD_PROTO);
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/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 6604517..def3059 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
@@ -722,13 +722,13 @@ public class BlockManager implements BlockStatsMXBean {
     mxBeanName = MBeans.register("NameNode", "BlockStats", this);
     bmSafeMode.activate(blockTotal);
     if (sps != null && !haEnabled) {
-      sps.start();
+      sps.start(false);
     }
   }
 
   public void close() {
     if (sps != null) {
-      sps.stop();
+      sps.stop(false);
     }
     bmSafeMode.close();
     try {
@@ -4975,7 +4975,7 @@ public class BlockManager implements BlockStatsMXBean {
       return;
     }
 
-    sps.start();
+    sps.start(true);
   }
 
   /**
@@ -4989,12 +4989,7 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is already stopped.");
       return;
     }
-    sps.stop();
-    // TODO: add command to DNs for stop in-progress processing SPS commands?
-    // to avoid confusions in cluster, I think sending commands from centralized
-    // place would be better to drop pending queues at DN. Anyway in progress
-    // work will be finished in a while, but this command can void starting
-    // fresh movements at DN.
+    sps.stop(true);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 9d3b544..95cb3a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -213,6 +213,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
   private final Queue<BlockStorageMovementInfosBatch> storageMovementBlocks =
       new LinkedList<>();
+  private volatile boolean dropSPSWork = false;
 
   /* Variables for maintaining number of blocks scheduled to be written to
    * this storage. This count is approximate and might be slightly bigger
@@ -1054,4 +1055,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
       return storageMovementBlocks.poll();
     }
   }
+
+  /**
+   * Set whether to drop SPS related queues at DN side.
+   *
+   * @param dropSPSWork
+   *          - true if need to drop SPS queues, otherwise false.
+   */
+  public synchronized void setDropSPSWork(boolean dropSPSWork) {
+    this.dropSPSWork = dropSPSWork;
+  }
+
+  /**
+   * @return true if need to drop SPS queues at DN.
+   */
+  public synchronized boolean shouldDropSPSWork() {
+    return this.dropSPSWork;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 048b68f..51c5aef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1748,6 +1748,13 @@ public class DatanodeManager {
           blkStorageMovementInfosBatch.getBlockMovingInfo()));
     }
 
+    if (nodeinfo.shouldDropSPSWork()) {
+      cmds.add(DropSPSWorkCommand.DNA_DROP_SPS_WORK_COMMAND);
+      // Set back to false to indicate that the new value has been sent to the
+      // datanode.
+      nodeinfo.setDropSPSWork(false);
+    }
+
     if (!cmds.isEmpty()) {
       return cmds.toArray(new DatanodeCommand[cmds.size()]);
     }
@@ -1976,5 +1983,17 @@ public class DatanodeManager {
     return slowDiskTracker != null ?
         slowDiskTracker.getSlowDiskReportAsJsonString() : null;
   }
+
+  /**
+   * Mark all DNs to drop SPS queues. A DNA_DROP_SPS_WORK_COMMAND will be added
+   * in heartbeat response, which will indicate DN to drop SPS queues
+   */
+  public void addDropSPSWorkCommandsToAllDNs() {
+    synchronized (this) {
+      for (DatanodeDescriptor dn : datanodeMap.values()) {
+        dn.setDropSPSWork(true);
+      }
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index c77fe2b..39ff4b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -788,6 +788,10 @@ class BPOfferService {
           blkSPSCmd.getTrackID(), blkSPSCmd.getBlockPoolId(),
           blkSPSCmd.getBlockMovingTasks());
       break;
+    case DatanodeProtocol.DNA_DROP_SPS_WORK_COMMAND:
+      LOG.info("DatanodeCommand action: DNA_DROP_SPS_WORK_COMMAND");
+      dn.getStoragePolicySatisfyWorker().dropSPSWork();
+      break;
     default:
       LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
index bd35b09..e623cef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -146,4 +146,16 @@ public class BlockStorageMovementTracker implements Runnable {
       moverTaskFutures.notify();
     }
   }
+
+  /**
+   * Clear the pending movement and movement result queues.
+   */
+  void removeAll() {
+    synchronized (moverTaskFutures) {
+      moverTaskFutures.clear();
+    }
+    synchronized (movementResults) {
+      movementResults.clear();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 10adbfd..a96ac98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -115,7 +115,6 @@ public class StoragePolicySatisfyWorker {
         TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
         new Daemon.DaemonFactory() {
           private final AtomicInteger threadIndex = new AtomicInteger(0);
-
           @Override
           public Thread newThread(Runnable r) {
             Thread t = super.newThread(r);
@@ -421,10 +420,31 @@ public class StoragePolicySatisfyWorker {
         }
       }
     }
+
+    /**
+     * Clear the trackID vs movement status tracking map.
+     */
+    void removeAll() {
+      synchronized (trackIdVsMovementStatus) {
+        trackIdVsMovementStatus.clear();
+      }
+    }
+
   }
 
   @VisibleForTesting
   BlocksMovementsCompletionHandler getBlocksMovementsCompletionHandler() {
     return handler;
   }
+
+  /**
+   * Drop the in-progress SPS work queues.
+   */
+  public void dropSPSWork() {
+    LOG.info("Received request to drop StoragePolicySatisfierWorker queues. "
+        + "So, none of the SPS Worker queued block movements will"
+        + " be scheduled.");
+    movementTracker.removeAll();
+    handler.removeAll();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/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 1c48910..dc58294 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
@@ -99,9 +99,14 @@ public class StoragePolicySatisfier implements Runnable {
    * Start storage policy satisfier demon thread. Also start block storage
    * movements monitor for retry the attempts if needed.
    */
-  public synchronized void start() {
+  public synchronized void start(boolean reconfigStart) {
     isRunning = true;
-    LOG.info("Starting StoragePolicySatisfier.");
+    if (reconfigStart) {
+      LOG.info("Starting StoragePolicySatisfier, as admin requested to "
+          + "activate it.");
+    } else {
+      LOG.info("Starting StoragePolicySatisfier.");
+    }
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
@@ -110,10 +115,17 @@ public class StoragePolicySatisfier implements Runnable {
 
   /**
    * Stop storage policy satisfier demon thread.
+   *
+   * @param reconfigStop
    */
-  public synchronized void stop() {
+  public synchronized void stop(boolean reconfigStop) {
     isRunning = false;
-    LOG.info("Stopping StoragePolicySatisfier.");
+    if (reconfigStop) {
+      LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
+          + "deactivate it.");
+    } else {
+      LOG.info("Stopping StoragePolicySatisfier.");
+    }
     if (storagePolicySatisfierThread == null) {
       return;
     }
@@ -123,7 +135,10 @@ public class StoragePolicySatisfier implements Runnable {
     } catch (InterruptedException ie) {
     }
     this.storageMovementsMonitor.stop();
-    this.clearQueues();
+    if (reconfigStop) {
+      this.clearQueues();
+      this.blockManager.getDatanodeManager().addDropSPSWorkCommandsToAllDNs();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 858f59b..892efb3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -80,6 +80,8 @@ public interface DatanodeProtocol {
   final static int DNA_UNCACHE = 10;   // uncache blocks
   final static int DNA_ERASURE_CODING_RECONSTRUCTION = 11; // erasure coding reconstruction command
   final static int DNA_BLOCK_STORAGE_MOVEMENT = 12; // block storage movement command
+  final static int DNA_DROP_SPS_WORK_COMMAND = 13; // block storage movement
+                                                   // command
 
   /** 
    * Register Datanode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DropSPSWorkCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DropSPSWorkCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DropSPSWorkCommand.java
new file mode 100644
index 0000000..806f713
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DropSPSWorkCommand.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A DropSPSWorkCommand is an instruction to a datanode to drop the SPSWorker's
+ * pending block storage movement queues.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DropSPSWorkCommand extends DatanodeCommand {
+  public static final DropSPSWorkCommand DNA_DROP_SPS_WORK_COMMAND =
+      new DropSPSWorkCommand();
+
+  public DropSPSWorkCommand() {
+    super(DatanodeProtocol.DNA_DROP_SPS_WORK_COMMAND);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 77b0f86..899dc7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -61,6 +61,7 @@ message DatanodeCommandProto {
     BlockIdCommand = 8;
     BlockECReconstructionCommand = 9;
     BlockStorageMovementCommand = 10;
+    DropSPSWorkCommand = 11;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -76,6 +77,7 @@ message DatanodeCommandProto {
   optional BlockIdCommandProto blkIdCmd = 8;
   optional BlockECReconstructionCommandProto blkECReconstructionCmd = 9;
   optional BlockStorageMovementCommandProto blkStorageMovementCmd = 10;
+  optional DropSPSWorkCommandProto dropSPSWorkCmd = 11;
 }
 
 /**
@@ -166,6 +168,13 @@ message BlockStorageMovementCommandProto {
 }
 
 /**
+ * Instruct datanode to drop SPS work queues
+ */
+message DropSPSWorkCommandProto {
+  // void
+}
+
+/**
  * Block storage movement information
  */
 message BlockStorageMovementProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18d9bc52/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 8e02d41..86b8b50 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -186,6 +188,63 @@ public class TestStoragePolicySatisfyWorker {
     waitForBlockMovementCompletion(worker, inode.getId(), 1, 30000);
   }
 
+  /**
+   * Tests that drop SPS work method clears all the queues.
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testDropSPSWork() throws Exception {
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(20).build();
+
+    cluster.waitActive();
+    final DistributedFileSystem dfs = cluster.getFileSystem();
+    final String file = "/testDropSPSWork";
+    DFSTestUtil.createFile(dfs, new Path(file), false, 1024, 50 * 100,
+        DEFAULT_BLOCK_SIZE, (short) 2, 0, false, null);
+
+    // move to ARCHIVE
+    dfs.setStoragePolicy(new Path(file), "COLD");
+
+    DataNode src = cluster.getDataNodes().get(2);
+    DatanodeInfo targetDnInfo =
+        DFSTestUtil.getLocalDatanodeInfo(src.getXferPort());
+
+    StoragePolicySatisfyWorker worker =
+        new StoragePolicySatisfyWorker(conf, src);
+    List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+    List<LocatedBlock> locatedBlocks =
+        dfs.getClient().getLocatedBlocks(file, 0).getLocatedBlocks();
+    for (LocatedBlock locatedBlock : locatedBlocks) {
+      BlockMovingInfo blockMovingInfo =
+          prepareBlockMovingInfo(locatedBlock.getBlock().getLocalBlock(),
+              locatedBlock.getLocations()[0], targetDnInfo,
+              locatedBlock.getStorageTypes()[0], StorageType.ARCHIVE);
+      blockMovingInfos.add(blockMovingInfo);
+    }
+    INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
+    worker.processBlockMovingTasks(inode.getId(),
+        cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
+    // Wait till results queue build up
+    waitForBlockMovementResult(worker, inode.getId(), 30000);
+    worker.dropSPSWork();
+    assertTrue(worker.getBlocksMovementsCompletionHandler()
+        .getBlksMovementResults().size() == 0);
+  }
+
+  private void waitForBlockMovementResult(
+      final StoragePolicySatisfyWorker worker, final long inodeId, int timeout)
+          throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        List<BlocksStorageMovementResult> completedBlocks = worker
+            .getBlocksMovementsCompletionHandler().getBlksMovementResults();
+        return completedBlocks.size() > 0;
+      }
+    }, 100, timeout);
+  }
+
   private void waitForBlockMovementCompletion(
       final StoragePolicySatisfyWorker worker, final long inodeId,
       int expectedFailedItemsCount, int timeout) throws Exception {


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


[39/50] [abbrv] hadoop git commit: HDFS-12152: [SPS]: Re-arrange StoragePolicySatisfyWorker stopping sequence to improve thread cleanup time. Contributed by Rakesh R.

Posted by ra...@apache.org.
HDFS-12152: [SPS]: Re-arrange StoragePolicySatisfyWorker stopping sequence to improve thread cleanup time. 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/ad422f1b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ad422f1b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ad422f1b

Branch: refs/heads/HDFS-10285
Commit: ad422f1b312a8abc03ce82492a42505e3c2d2a77
Parents: c2c9296
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Jul 19 00:55:26 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:47 2017 +0530

----------------------------------------------------------------------
 .../datanode/BlockStorageMovementTracker.java       | 16 ++++++++++++----
 .../server/datanode/StoragePolicySatisfyWorker.java |  5 +++--
 2 files changed, 15 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad422f1b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
index c7e952b..f3d2bb6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -77,7 +77,8 @@ public class BlockStorageMovementTracker implements Runnable {
             moverTaskFutures.wait(2000);
           }
         } catch (InterruptedException ignore) {
-          // ignore
+          // Sets interrupt flag of this thread.
+          Thread.currentThread().interrupt();
         }
       }
       try {
@@ -102,12 +103,19 @@ public class BlockStorageMovementTracker implements Runnable {
             synchronized (moverTaskFutures) {
               moverTaskFutures.remove(trackId);
             }
-            // handle completed or inprogress blocks movements per trackId.
-            blksMovementsStatusHandler.handle(resultPerTrackIdList);
+            if (running) {
+              // handle completed or inprogress blocks movements per trackId.
+              blksMovementsStatusHandler.handle(resultPerTrackIdList);
+            }
             movementResults.remove(trackId);
           }
         }
-      } catch (ExecutionException | InterruptedException e) {
+      } catch (InterruptedException e) {
+        if (running) {
+          LOG.error("Exception while moving block replica to target storage"
+              + " type", e);
+        }
+      } catch (ExecutionException e) {
         // TODO: Do we need failure retries and implement the same if required.
         LOG.error("Exception while moving block replica to target storage type",
             e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad422f1b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 196cd58..4e57805 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -137,8 +137,8 @@ public class StoragePolicySatisfyWorker {
    * thread.
    */
   void stop() {
-    movementTrackerThread.interrupt();
     movementTracker.stopTracking();
+    movementTrackerThread.interrupt();
   }
 
   /**
@@ -147,7 +147,8 @@ public class StoragePolicySatisfyWorker {
   void waitToFinishWorkerThread() {
     try {
       movementTrackerThread.join(3000);
-    } catch (InterruptedException ie) {
+    } catch (InterruptedException ignore) {
+      // ignore
     }
   }
 


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


[20/50] [abbrv] hadoop git commit: HDFS-11186. [SPS]: Daemon thread of SPS should start only in Active NN. Contributed by Wei Zhou

Posted by ra...@apache.org.
HDFS-11186. [SPS]: Daemon thread of SPS should start only in Active NN. Contributed by Wei Zhou


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

Branch: refs/heads/HDFS-10285
Commit: 66bf0e6753023650e6b1268b5220806b8d0aa2c3
Parents: 57f0b2f
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Thu Jan 12 09:01:30 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:57 2017 +0530

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |   9 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |  11 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   6 +
 .../hadoop/hdfs/server/namenode/NameNode.java   |   7 ++
 .../TestStoragePolicySatisfierWithHA.java       | 109 +++++++++++++++++++
 5 files changed, 138 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/66bf0e67/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 472a537..6604517 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
@@ -442,9 +442,15 @@ 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);
@@ -715,7 +721,7 @@ public class BlockManager implements BlockStatsMXBean {
     this.blockReportThread.start();
     mxBeanName = MBeans.register("NameNode", "BlockStats", this);
     bmSafeMode.activate(blockTotal);
-    if (sps != null) {
+    if (sps != null && !haEnabled) {
       sps.start();
     }
   }
@@ -4968,6 +4974,7 @@ public class BlockManager implements BlockStatsMXBean {
       LOG.info("Storage policy satisfier is already running.");
       return;
     }
+
     sps.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66bf0e67/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 6edf110..3b76fe9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
@@ -661,9 +662,13 @@ public class Mover {
           try {
             spsRunning = nnc.getDistributedFileSystem().getClient()
                 .isStoragePolicySatisfierRunning();
-          } catch (StandbyException e) {
-            System.err.println("Skip Standby Namenode. " + nnc.toString());
-            continue;
+          } catch (RemoteException e) {
+            IOException cause = e.unwrapRemoteException();
+            if (cause instanceof StandbyException) {
+              System.err.println("Skip Standby Namenode. " + nnc.toString());
+              continue;
+            }
+            throw e;
           }
           if (spsRunning) {
             System.err.println("Mover failed due to StoragePolicySatisfier"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66bf0e67/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 ae38685..2478466 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
@@ -1284,6 +1284,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         FSDirEncryptionZoneOp.warmUpEdekCache(edekCacheLoader, dir,
             edekCacheLoaderDelay, edekCacheLoaderInterval);
       }
+
+      blockManager.activateSPS();
     } finally {
       startingActiveService = false;
       blockManager.checkSafeMode();
@@ -1313,6 +1315,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     LOG.info("Stopping services started for active state");
     writeLock();
     try {
+      if (blockManager != null) {
+        blockManager.deactivateSPS();
+      }
+
       stopSecretManager();
       leaseManager.stopMonitor();
       if (nnrmthread != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66bf0e67/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 4f1423a..63eaa3a 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
@@ -2148,6 +2148,13 @@ public class NameNode extends ReconfigurableBase implements
                   + "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 "
+              + state + " NameNode is not allowed"));
+    }
+
     boolean activateSPS = Boolean.parseBoolean(newVal);
     if (activateSPS) {
       namesystem.getBlockManager().activateSPS();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66bf0e67/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
new file mode 100644
index 0000000..4d226ff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfierWithHA.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.ReconfigurationException;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Tests that StoragePolicySatisfier is able to work with HA enabled.
+ */
+public class TestStoragePolicySatisfierWithHA {
+  private MiniDFSCluster cluster = null;
+
+  @Before
+  public void setUp() throws IOException {
+    Configuration conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(1)
+        .build();
+  }
+
+  /**
+   * Tests to verify that SPS should run/stop automatically when NN state
+   * changes between Standby and Active.
+   */
+  @Test(timeout = 100000)
+  public void testWhenNNHAStateChanges() throws IOException {
+    try {
+      DistributedFileSystem fs;
+      boolean running;
+
+      cluster.waitActive();
+      fs = cluster.getFileSystem(0);
+
+      try {
+        fs.getClient().isStoragePolicySatisfierRunning();
+        Assert.fail("Call this function to Standby NN should "
+            + "raise an exception.");
+      } catch (RemoteException e) {
+        IOException cause = e.unwrapRemoteException();
+        if (!(cause instanceof StandbyException)) {
+          Assert.fail("Unexpected exception happened " + e);
+        }
+      }
+
+      cluster.transitionToActive(0);
+      running = fs.getClient().isStoragePolicySatisfierRunning();
+      Assert.assertTrue("StoragePolicySatisfier should be active "
+          + "when NN transits from Standby to Active mode.", running);
+
+      // NN transits from Active to Standby
+      cluster.transitionToStandby(0);
+      try {
+        fs.getClient().isStoragePolicySatisfierRunning();
+        Assert.fail("NN in Standby again, call this function should "
+            + "raise an exception.");
+      } catch (RemoteException e) {
+        IOException cause = e.unwrapRemoteException();
+        if (!(cause instanceof StandbyException)) {
+          Assert.fail("Unexpected exception happened " + e);
+        }
+      }
+
+      try {
+        cluster.getNameNode(0).reconfigurePropertyImpl(
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, "false");
+        Assert.fail("It's not allowed to activate or deactivate"
+            + " StoragePolicySatisfier on Standby NameNode");
+      } catch (ReconfigurationException e) {
+        GenericTestUtils.assertExceptionContains("Could not change property "
+            + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY
+            + " from 'true' to 'false'", e);
+        GenericTestUtils.assertExceptionContains(
+            "Activating or deactivating storage policy satisfier service on "
+                + "standby NameNode is not allowed", e.getCause());
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}


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


[12/50] [abbrv] hadoop git commit: HDFS-10885. [SPS]: Mover tool should not be allowed to run when Storage Policy Satisfier is on. Contributed by Wei Zhou

Posted by ra...@apache.org.
HDFS-10885. [SPS]: Mover tool should not be allowed to run when Storage Policy Satisfier is on. Contributed by Wei Zhou


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

Branch: refs/heads/HDFS-10285
Commit: aca6ab69a9e6cd05f0d7c828ed0c42887b03bdf9
Parents: a9adc9d
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Tue Dec 6 17:56:08 2016 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:26:01 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  4 ++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  8 +++
 .../ClientNamenodeProtocolTranslatorPB.java     | 18 ++++++
 .../src/main/proto/ClientNamenodeProtocol.proto |  9 +++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  5 ++
 ...tNamenodeProtocolServerSideTranslatorPB.java | 18 ++++++
 .../hadoop/hdfs/server/balancer/ExitStatus.java |  3 +-
 .../server/blockmanagement/BlockManager.java    | 27 ++++++++-
 .../hdfs/server/common/HdfsServerConstants.java |  3 +
 .../apache/hadoop/hdfs/server/mover/Mover.java  | 23 ++++++-
 .../hdfs/server/namenode/FSNamesystem.java      |  6 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java | 11 ++++
 .../server/namenode/StoragePolicySatisfier.java | 37 +++++++++++
 .../src/main/resources/hdfs-default.xml         |  9 +++
 .../TestStoragePolicySatisfyWorker.java         |  2 +
 .../hadoop/hdfs/server/mover/TestMover.java     | 64 ++++++++++++++++++--
 .../hdfs/server/mover/TestStorageMover.java     |  2 +
 .../namenode/TestStoragePolicySatisfier.java    | 23 +++++++
 18 files changed, 258 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 7f204f7..376ff60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -3036,6 +3036,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public boolean isStoragePolicySatisfierRunning() throws IOException {
+    return namenode.isStoragePolicySatisfierRunning();
+  }
+
   Tracer getTracer() {
     return tracer;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index b1affc5..ae19e27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1713,4 +1713,12 @@ public interface ClientProtocol {
    */
   @Idempotent
   void satisfyStoragePolicy(String path) throws IOException;
+
+  /**
+   * Check if StoragePolicySatisfier is running.
+   * @return true if StoragePolicySatisfier is running
+   * @throws IOException
+   */
+  @Idempotent
+  boolean isStoragePolicySatisfierRunning() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index cede194..269d095 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -142,6 +142,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSto
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsStoragePolicySatisfierRunningRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsStoragePolicySatisfierRunningResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
@@ -284,6 +286,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
   private final static GetErasureCodingCodecsRequestProto
       VOID_GET_EC_CODEC_REQUEST = GetErasureCodingCodecsRequestProto
       .newBuilder().build();
+  private final static IsStoragePolicySatisfierRunningRequestProto
+      VOID_IS_SPS_RUNNING_REQUEST = IsStoragePolicySatisfierRunningRequestProto
+      .newBuilder().build();
+
 
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
@@ -1860,6 +1866,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
+  public boolean isStoragePolicySatisfierRunning() throws IOException {
+    try {
+      IsStoragePolicySatisfierRunningResponseProto rep =
+          rpcProxy.isStoragePolicySatisfierRunning(null,
+              VOID_IS_SPS_RUNNING_REQUEST);
+      return rep.getRunning();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public QuotaUsage getQuotaUsage(String path) throws IOException {
     GetQuotaUsageRequestProto req =
         GetQuotaUsageRequestProto.newBuilder().setPath(path).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 23a5da6..68a6124 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -811,6 +811,13 @@ message SatisfyStoragePolicyResponseProto {
 
 }
 
+message IsStoragePolicySatisfierRunningRequestProto {  // no parameters
+}
+
+message IsStoragePolicySatisfierRunningResponseProto {
+  required bool running = 1;
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -995,4 +1002,6 @@ service ClientNamenodeProtocol {
       returns(ListOpenFilesResponseProto);
   rpc satisfyStoragePolicy(SatisfyStoragePolicyRequestProto)
       returns(SatisfyStoragePolicyResponseProto);
+  rpc isStoragePolicySatisfierRunning(IsStoragePolicySatisfierRunningRequestProto)
+      returns(IsStoragePolicySatisfierRunningResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 97b8b1a..288cc2b 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
@@ -569,6 +569,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_MOVER_MAX_NO_MOVE_INTERVAL_KEY = "dfs.mover.max-no-move-interval";
   public static final int    DFS_MOVER_MAX_NO_MOVE_INTERVAL_DEFAULT = 60*1000; // One minute
 
+  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_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 9866;
   public static final String  DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_DEFAULT_PORT;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index b1f68b3..df16f5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -154,6 +154,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSto
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsStoragePolicySatisfierRunningRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsStoragePolicySatisfierRunningResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
@@ -1820,6 +1822,22 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public IsStoragePolicySatisfierRunningResponseProto
+      isStoragePolicySatisfierRunning(RpcController controller,
+      IsStoragePolicySatisfierRunningRequestProto req)
+      throws ServiceException {
+    try {
+      boolean ret = server.isStoragePolicySatisfierRunning();
+      IsStoragePolicySatisfierRunningResponseProto.Builder builder =
+          IsStoragePolicySatisfierRunningResponseProto.newBuilder();
+      builder.setRunning(ret);
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public GetQuotaUsageResponseProto getQuotaUsage(
       RpcController controller, GetQuotaUsageRequestProto req)
       throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/ExitStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/ExitStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/ExitStatus.java
index 6bf2986..5cf4204 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/ExitStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/ExitStatus.java
@@ -30,7 +30,8 @@ public enum ExitStatus {
   IO_EXCEPTION(-4),
   ILLEGAL_ARGUMENTS(-5),
   INTERRUPTED(-6),
-  UNFINALIZED_UPGRADE(-7);
+  UNFINALIZED_UPGRADE(-7),
+  SKIPPED_DUE_TO_SPS(-8);
 
   private final int code;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 a0fe450..1c2b0a2 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
@@ -472,7 +472,24 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
         * 1000L);
 
-    sps = new StoragePolicySatisfier(namesystem, storageMovementNeeded, this);
+    final boolean storagePolicyEnabled =
+        conf.getBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY,
+            DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT);
+    final boolean 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);
+    } 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);
+    }
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     this.maxCorruptFilesReturned = conf.getInt(
@@ -698,11 +715,15 @@ public class BlockManager implements BlockStatsMXBean {
     this.blockReportThread.start();
     mxBeanName = MBeans.register("NameNode", "BlockStats", this);
     bmSafeMode.activate(blockTotal);
-    sps.start();
+    if (sps != null) {
+      sps.start();
+    }
   }
 
   public void close() {
-    sps.stop();
+    if (sps != null) {
+      sps.stop();
+    }
     bmSafeMode.close();
     try {
       redundancyThread.interrupt();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index c3098f3..e486317 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -24,6 +24,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
@@ -364,6 +365,8 @@ public interface HdfsServerConstants {
   String XATTR_ERASURECODING_POLICY =
       "system.hdfs.erasurecoding.policy";
 
+  Path MOVER_ID_PATH = new Path("/system/mover.id");
+
   long BLOCK_GROUP_INDEX_MASK = 15;
   byte MAX_BLOCKS_IN_GROUP = 16;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 8b89378..6edf110 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -41,11 +41,14 @@ import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.Matcher;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.SecurityUtil;
@@ -70,8 +73,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 public class Mover {
   static final Log LOG = LogFactory.getLog(Mover.class);
 
-  static final Path MOVER_ID_PATH = new Path("/system/mover.id");
-
   private static class StorageMap {
     private final StorageGroupMap<Source> sources
         = new StorageGroupMap<Source>();
@@ -645,7 +646,7 @@ public class Mover {
     List<NameNodeConnector> connectors = Collections.emptyList();
     try {
       connectors = NameNodeConnector.newNameNodeConnectors(namenodes,
-          Mover.class.getSimpleName(), MOVER_ID_PATH, conf,
+          Mover.class.getSimpleName(), HdfsServerConstants.MOVER_ID_PATH, conf,
           NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
 
       while (connectors.size() > 0) {
@@ -655,6 +656,22 @@ public class Mover {
           NameNodeConnector nnc = iter.next();
           final Mover m = new Mover(nnc, conf, retryCount,
               excludedPinnedBlocks);
+
+          boolean spsRunning;
+          try {
+            spsRunning = nnc.getDistributedFileSystem().getClient()
+                .isStoragePolicySatisfierRunning();
+          } catch (StandbyException e) {
+            System.err.println("Skip Standby Namenode. " + nnc.toString());
+            continue;
+          }
+          if (spsRunning) {
+            System.err.println("Mover failed due to StoragePolicySatisfier"
+                + " is running. Exiting with status "
+                + ExitStatus.SKIPPED_DUE_TO_SPS + "... ");
+            return ExitStatus.SKIPPED_DUE_TO_SPS.getExitCode();
+          }
+
           final ExitStatus r = m.run();
 
           if (r == ExitStatus.SUCCESS) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 48b60e7..ec7aafc 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
@@ -3792,8 +3792,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       // TODO: Handle blocks movement results send by the coordinator datanode.
       // This has to be revisited as part of HDFS-11029.
-      blockManager.getStoragePolicySatisfier()
-          .handleBlocksStorageMovementResults(blksMovementResults);
+      if (blockManager.getStoragePolicySatisfier() != null) {
+        blockManager.getStoragePolicySatisfier()
+            .handleBlocksStorageMovementResults(blksMovementResults);
+      }
 
       //create ha status
       final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index e3173e5..a7cf273 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -2466,4 +2466,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.logAuditEvent(true, operationName, null);
     return result;
   }
+
+  @Override
+  public boolean isStoragePolicySatisfierRunning() throws IOException {
+    checkNNStartup();
+    if (nn.isStandbyState()) {
+      throw new StandbyException("Not supported by Standby Namenode.");
+    }
+    StoragePolicySatisfier sps = namesystem.getBlockManager()
+        .getStoragePolicySatisfier();
+    return sps != null && sps.isRunning();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 617ab2c..cc2ca7d 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -70,6 +72,7 @@ public class StoragePolicySatisfier implements Runnable {
   private final BlockManager blockManager;
   private final BlockStorageMovementNeeded storageMovementNeeded;
   private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
+  private volatile boolean isRunning = false;
 
   public StoragePolicySatisfier(final Namesystem namesystem,
       final BlockStorageMovementNeeded storageMovementNeeded,
@@ -99,6 +102,7 @@ public class StoragePolicySatisfier implements Runnable {
    * Stop storage policy satisfier demon thread.
    */
   public void stop() {
+    isRunning = false;
     if (storagePolicySatisfierThread == null) {
       return;
     }
@@ -110,8 +114,40 @@ public class StoragePolicySatisfier implements Runnable {
     this.storageMovementsMonitor.stop();
   }
 
+  /**
+   * Check whether StoragePolicySatisfier is running.
+   * @return true if running
+   */
+  public boolean isRunning() {
+    return isRunning;
+  }
+
+  // Return true if a Mover instance is running
+  private boolean checkIfMoverRunning() {
+    boolean ret = false;
+    try {
+      String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
+      INode inode = namesystem.getFSDirectory().getINode(
+          moverId, FSDirectory.DirOp.READ);
+      if (inode != null) {
+        ret = true;
+      }
+    } catch (IOException e) {
+      LOG.info("StoragePolicySatisfier is enabled as no Mover ID file found.");
+      ret = false;
+    }
+    return ret;
+  }
+
   @Override
   public void run() {
+    isRunning = !checkIfMoverRunning();
+    if (!isRunning) {
+      LOG.error("StoragePolicySatisfier thread stopped "
+          + "as Mover ID file " + HdfsServerConstants.MOVER_ID_PATH.toString()
+          + " exists");
+      return;
+    }
     while (namesystem.isRunning()) {
       try {
         Long blockCollectionID = storageMovementNeeded.get();
@@ -123,6 +159,7 @@ public class StoragePolicySatisfier implements Runnable {
         // we want to check block movements.
         Thread.sleep(3000);
       } catch (Throwable t) {
+        isRunning = false;
         if (!namesystem.isRunning()) {
           LOG.info("Stopping StoragePolicySatisfier.");
           if (!(t instanceof InterruptedException)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 dedf987..99ac219 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
@@ -4344,6 +4344,15 @@
 </property>
 
 <property>
+  <name>dfs.storage.policy.satisfier.activate</name>
+  <value>true</value>
+  <description>
+    If true, activate StoragePolicySatisfier.
+    By default, StoragePolicySatisfier is activated.
+  </description>
+</property>
+
+<property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 1eb44e0..8e02d41 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
@@ -67,6 +67,8 @@ 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,
+        true);
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 461da07..de7dd29 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
@@ -77,6 +77,7 @@ import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.InternalDataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.mover.Mover.MLocation;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.http.HttpConfig;
@@ -112,6 +113,8 @@ public class TestMover {
     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, false);
   }
 
   static Mover newMover(Configuration conf) throws IOException {
@@ -123,7 +126,7 @@ public class TestMover {
     }
 
     final List<NameNodeConnector> nncs = NameNodeConnector.newNameNodeConnectors(
-        nnMap, Mover.class.getSimpleName(), Mover.MOVER_ID_PATH, conf,
+        nnMap, Mover.class.getSimpleName(), HdfsServerConstants.MOVER_ID_PATH, conf,
         NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
     return new Mover(nncs.get(0), conf, new AtomicInteger(0), new HashMap<>());
   }
@@ -131,6 +134,8 @@ public class TestMover {
   @Test
   public void testScheduleSameBlock() throws IOException {
     final Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(4).build();
     try {
@@ -245,8 +250,11 @@ public class TestMover {
    */
   @Test
   public void testMoverCli() throws Exception {
+    final Configuration clusterConf = new HdfsConfiguration();
+    clusterConf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
-        .Builder(new HdfsConfiguration()).numDataNodes(0).build();
+        .Builder(clusterConf).numDataNodes(0).build();
     try {
       final Configuration conf = cluster.getConfiguration(0);
       try {
@@ -278,8 +286,10 @@ public class TestMover {
   @Test
   public void testMoverCliWithHAConf() throws Exception {
     final Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
-        .Builder(new HdfsConfiguration())
+        .Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
         .numDataNodes(0).build();
     HATestUtil.setFailoverConfigurations(cluster, conf, "MyCluster");
@@ -300,11 +310,16 @@ public class TestMover {
 
   @Test
   public void testMoverCliWithFederation() throws Exception {
+    final Configuration clusterConf = new HdfsConfiguration();
+    clusterConf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
-        .Builder(new HdfsConfiguration())
+        .Builder(clusterConf)
         .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
         .numDataNodes(0).build();
     final Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     DFSTestUtil.setFederatedConfiguration(cluster, conf);
     try {
       Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -348,11 +363,16 @@ public class TestMover {
 
   @Test
   public void testMoverCliWithFederationHA() throws Exception {
+    final Configuration clusterConf = new HdfsConfiguration();
+    clusterConf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster
-        .Builder(new HdfsConfiguration())
+        .Builder(clusterConf)
         .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(3))
         .numDataNodes(0).build();
     final Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     DFSTestUtil.setFederatedHAConfiguration(cluster, conf);
     try {
       Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
@@ -416,6 +436,8 @@ public class TestMover {
   public void testMoveWhenStoragePolicyNotSatisfying() throws Exception {
     // HDFS-8147
     final Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(3)
         .storageTypes(
@@ -441,6 +463,36 @@ public class TestMover {
     }
   }
 
+  @Test(timeout = 300000)
+  public void testMoveWhenStoragePolicySatisfierIsRunning() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, true);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK}, {StorageType.DISK},
+                {StorageType.DISK}}).build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testMoveWhenStoragePolicySatisfierIsRunning";
+      // write to DISK
+      final FSDataOutputStream out = dfs.create(new Path(file));
+      out.writeChars("testMoveWhenStoragePolicySatisfierIsRunning");
+      out.close();
+
+      // move to ARCHIVE
+      dfs.setStoragePolicy(new Path(file), "COLD");
+      int rc = ToolRunner.run(conf, new Mover.Cli(),
+          new String[] {"-p", file.toString()});
+      int exitcode = ExitStatus.SKIPPED_DUE_TO_SPS.getExitCode();
+      Assert.assertEquals("Exit code should be " + exitcode, exitcode, rc);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   public void testMoverFailedRetry() throws Exception {
     // HDFS-8147
@@ -491,6 +543,8 @@ public class TestMover {
         1L);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
+    conf.setBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false);
   }
 
   @Test(timeout = 300000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 356ae3a..d548649 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
@@ -96,6 +96,8 @@ public class TestStorageMover {
     DEFAULT_CONF.setLong(
         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);
 
     DEFAULT_POLICIES = BlockStoragePolicySuite.createDefaultSuite();
     HOT = DEFAULT_POLICIES.getPolicy(HdfsConstants.HOT_STORAGE_POLICY_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aca6ab69/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 499fe3c..fe23f3e 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
@@ -31,12 +31,14 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -442,6 +444,27 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Tests to verify that SPS should not start when a Mover instance
+   * is running.
+   */
+  @Test(timeout = 300000)
+  public void testWhenMoverIsAlreadyRunningBeforeStoragePolicySatisfier()
+      throws IOException {
+    try {
+      // Simulate Mover by creating MOVER_ID file
+      DFSTestUtil.createFile(hdfsCluster.getFileSystem(),
+          HdfsServerConstants.MOVER_ID_PATH, 0, (short) 1, 0);
+      hdfsCluster.restartNameNode(true);
+      boolean running = hdfsCluster.getFileSystem()
+          .getClient().isStoragePolicySatisfierRunning();
+      Assert.assertFalse("SPS should not start "
+          + "when a Mover instance is running", running);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
   private void waitForAttemptedItems(long expectedBlkMovAttemptedCount,
       int timeout) throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();


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


[29/50] [abbrv] hadoop git commit: HDFS-11883: [SPS] : Handle NPE in BlockStorageMovementTracker when dropSPSWork() called. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-11883: [SPS] : Handle NPE in BlockStorageMovementTracker when dropSPSWork() called. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 8de5ddf082c0e6f345d0550017f9be13226f6b7c
Parents: 76da9b1
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue May 30 18:12:17 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:42 2017 +0530

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockStorageMovementTracker.java      | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8de5ddf0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
index 99858bc..c7e952b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -88,13 +88,17 @@ public class BlockStorageMovementTracker implements Runnable {
           long trackId = result.getTrackId();
           List<Future<BlockMovementResult>> blocksMoving = moverTaskFutures
               .get(trackId);
+          if (blocksMoving == null) {
+            LOG.warn("Future task doesn't exist for trackId " + trackId);
+            continue;
+          }
           blocksMoving.remove(future);
 
           List<BlockMovementResult> resultPerTrackIdList =
               addMovementResultToTrackIdList(result);
 
           // Completed all the scheduled blocks movement under this 'trackId'.
-          if (blocksMoving.isEmpty()) {
+          if (blocksMoving.isEmpty() || moverTaskFutures.get(trackId) == null) {
             synchronized (moverTaskFutures) {
               moverTaskFutures.remove(trackId);
             }


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


[50/50] [abbrv] hadoop git commit: HDFS-12106: [SPS]: Improve storage policy satisfier configurations. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-12106: [SPS]: Improve storage policy satisfier configurations. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 73f7db6193d11d288c1b89589da0929fa65a3559
Parents: bd059a1
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Wed Nov 15 20:22:27 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:31:34 2017 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  6 +++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  4 ++
 .../src/main/proto/ClientNamenodeProtocol.proto |  3 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   | 10 ++--
 .../server/blockmanagement/DatanodeManager.java | 12 ++---
 .../datanode/StoragePolicySatisfyWorker.java    |  3 +-
 .../BlockStorageMovementAttemptedItems.java     |  8 +--
 .../namenode/BlockStorageMovementNeeded.java    | 46 ++++++++++++----
 .../hdfs/server/namenode/FSNamesystem.java      |  3 ++
 .../server/namenode/StoragePolicySatisfier.java | 42 ++++++++++++---
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   | 27 +++++++---
 .../src/main/resources/hdfs-default.xml         | 17 ++++--
 .../src/site/markdown/ArchivalStorage.md        |  2 +-
 .../TestBlockStorageMovementAttemptedItems.java | 10 ++--
 .../namenode/TestStoragePolicySatisfier.java    | 57 ++++++++++++++++++--
 15 files changed, 199 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index aa0496e..c09c61c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -147,6 +147,12 @@ public final class HdfsConstants {
     SUCCESS,
 
     /**
+     * Few blocks failed to move and the path is still not
+     * fully satisfied the storage policy.
+     */
+    FAILURE,
+
+    /**
      * Status not available.
      */
     NOT_AVAILABLE

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index b396c3d..541f8cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -3249,6 +3249,8 @@ public class PBHelperClient {
       return StoragePolicySatisfyPathStatus.IN_PROGRESS;
     case SUCCESS:
       return StoragePolicySatisfyPathStatus.SUCCESS;
+    case FAILURE:
+      return StoragePolicySatisfyPathStatus.FAILURE;
     case NOT_AVAILABLE:
       return StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
     default:
@@ -3265,6 +3267,8 @@ public class PBHelperClient {
       return HdfsConstants.StoragePolicySatisfyPathStatus.IN_PROGRESS;
     case SUCCESS:
       return HdfsConstants.StoragePolicySatisfyPathStatus.SUCCESS;
+    case FAILURE:
+      return HdfsConstants.StoragePolicySatisfyPathStatus.FAILURE;
     case NOT_AVAILABLE:
       return HdfsConstants.StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
     default:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 0e7f779..24cef0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -826,7 +826,8 @@ message CheckStoragePolicySatisfyPathStatusResponseProto {
     PENDING = 0;
     IN_PROGRESS = 1;
     SUCCESS = 2;
-    NOT_AVAILABLE = 3;
+    FAILURE = 3;
+    NOT_AVAILABLE = 4;
   }
   required StoragePolicySatisfyPathStatus status = 1;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 a7a97a9..22a85f8 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
@@ -590,10 +590,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.storage.policy.satisfier.self.retry.timeout.millis";
   public static final int DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT =
       5 * 60 * 1000;
-  public static final String DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY =
+  public static final String DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY =
+      "dfs.storage.policy.satisfier.retry.max.attempts";
+  public static final int DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT =
+      3;
+  public static final String DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_KEY =
       "dfs.storage.policy.satisfier.low.max-streams.preference";
-  public static final boolean DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_DEFAULT =
-      false;
+  public static final boolean DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_DEFAULT =
+      true;
 
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 9866;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 4ea41d9..5cc4804 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -208,7 +208,7 @@ public class DatanodeManager {
    */
   private final long timeBetweenResendingCachingDirectivesMs;
 
-  private final boolean blocksToMoveShareEqualRatio;
+  private final boolean blocksToMoveLowPriority;
 
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
       final Configuration conf) throws IOException {
@@ -337,9 +337,9 @@ public class DatanodeManager {
 
     // SPS configuration to decide blocks to move can share equal ratio of
     // maxtransfers with pending replica and erasure-coded reconstruction tasks
-    blocksToMoveShareEqualRatio = conf.getBoolean(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY,
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_DEFAULT);
+    blocksToMoveLowPriority = conf.getBoolean(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_DEFAULT);
   }
 
   private static long getStaleIntervalFromConf(Configuration conf,
@@ -1697,11 +1697,11 @@ public class DatanodeManager {
       int numReplicationTasks = 0;
       int numECTasks = 0;
       int numBlocksToMoveTasks = 0;
-      // Check blocksToMoveShareEqualRatio configuration is true/false. If true,
+      // Check blocksToMoveLowPriority configuration is true/false. If false,
       // then equally sharing the max transfer. Otherwise gives high priority to
       // the pending_replica/erasure-coded tasks and only the delta streams will
       // be used for blocks to move tasks.
-      if (blocksToMoveShareEqualRatio) {
+      if (!blocksToMoveLowPriority) {
         // add blocksToMove count to total blocks so that will get equal share
         totalBlocks = totalBlocks + totalBlocksToMove;
         numReplicationTasks = (int) Math

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 47318f8..9a9c7e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -196,7 +196,8 @@ public class StoragePolicySatisfyWorker {
    * This class encapsulates the process of moving the block replica to the
    * given target and wait for the response.
    */
-  private class BlockMovingTask implements Callable<BlockMovementAttemptFinished> {
+  private class BlockMovingTask implements
+      Callable<BlockMovementAttemptFinished> {
     private final String blockPoolID;
     private final Block block;
     private final DatanodeInfo source;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 cc5b63a..643255f 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
@@ -183,7 +183,7 @@ public class BlockStorageMovementAttemptedItems {
           Long blockCollectionID = itemInfo.getTrackId();
           synchronized (movementFinishedBlocks) {
             ItemInfo candidate = new ItemInfo(itemInfo.getStartId(),
-                blockCollectionID);
+                blockCollectionID, itemInfo.getRetryCount() + 1);
             blockStorageMovementNeeded.add(candidate);
             iter.remove();
             LOG.info("TrackID: {} becomes timed out and moved to needed "
@@ -211,9 +211,9 @@ public class BlockStorageMovementAttemptedItems {
               // TODO: try add this at front of the Queue, so that this element
               // gets the chance first and can be cleaned from queue quickly as
               // all movements already done.
-              blockStorageMovementNeeded
-                  .add(new ItemInfo(attemptedItemInfo.getStartId(),
-                      attemptedItemInfo.getTrackId()));
+              blockStorageMovementNeeded.add(new ItemInfo(attemptedItemInfo
+                  .getStartId(), attemptedItemInfo.getTrackId(),
+                  attemptedItemInfo.getRetryCount() + 1));
               iterator.remove();
             }
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
index 8f7487c..89bcbff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
@@ -178,8 +178,8 @@ public class BlockStorageMovementNeeded {
    * Decrease the pending child count for directory once one file blocks moved
    * successfully. Remove the SPS xAttr if pending child count is zero.
    */
-  public synchronized void removeItemTrackInfo(ItemInfo trackInfo)
-      throws IOException {
+  public synchronized void removeItemTrackInfo(ItemInfo trackInfo,
+      boolean isSuccess) throws IOException {
     if (trackInfo.isDir()) {
       // If track is part of some start inode then reduce the pending
       // directory work count.
@@ -188,7 +188,7 @@ public class BlockStorageMovementNeeded {
       if (inode == null) {
         // directory deleted just remove it.
         this.pendingWorkForDirectory.remove(startId);
-        markSuccess(startId);
+        updateStatus(startId, isSuccess);
       } else {
         DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
         if (pendingWork != null) {
@@ -196,8 +196,10 @@ public class BlockStorageMovementNeeded {
           if (pendingWork.isDirWorkDone()) {
             namesystem.removeXattr(startId, XATTR_SATISFY_STORAGE_POLICY);
             pendingWorkForDirectory.remove(startId);
-            markSuccess(startId);
+            pendingWork.setFailure(!isSuccess);
+            updateStatus(startId, pendingWork.isPolicySatisfied());
           }
+          pendingWork.setFailure(isSuccess);
         }
       }
     } else {
@@ -205,7 +207,7 @@ public class BlockStorageMovementNeeded {
       // storageMovementAttemptedItems or file policy satisfied.
       namesystem.removeXattr(trackInfo.getTrackId(),
           XATTR_SATISFY_STORAGE_POLICY);
-      markSuccess(trackInfo.getStartId());
+      updateStatus(trackInfo.getStartId(), isSuccess);
     }
   }
 
@@ -224,14 +226,19 @@ public class BlockStorageMovementNeeded {
   /**
    * Mark inode status as SUCCESS in map.
    */
-  private void markSuccess(long startId){
+  private void updateStatus(long startId, boolean isSuccess){
     StoragePolicySatisfyPathStatusInfo spsStatusInfo =
         spsStatus.get(startId);
     if (spsStatusInfo == null) {
       spsStatusInfo = new StoragePolicySatisfyPathStatusInfo();
       spsStatus.put(startId, spsStatusInfo);
     }
-    spsStatusInfo.setSuccess();
+
+    if (isSuccess) {
+      spsStatusInfo.setSuccess();
+    } else {
+      spsStatusInfo.setFailure();
+    }
   }
 
   /**
@@ -325,7 +332,7 @@ public class BlockStorageMovementNeeded {
                   namesystem.removeXattr(startInode.getId(),
                       XATTR_SATISFY_STORAGE_POLICY);
                   pendingWorkForDirectory.remove(startInode.getId());
-                  markSuccess(startInode.getId());
+                  updateStatus(startInode.getId(), true);
                 }
               }
             }
@@ -431,6 +438,7 @@ public class BlockStorageMovementNeeded {
 
     private int pendingWorkCount = 0;
     private boolean fullyScanned = false;
+    private boolean success = true;
 
     /**
      * Increment the pending work count for directory.
@@ -461,6 +469,20 @@ public class BlockStorageMovementNeeded {
     public synchronized void markScanCompleted() {
       this.fullyScanned = true;
     }
+
+    /**
+     * Return true if all the files block movement is success, otherwise false.
+     */
+    public boolean isPolicySatisfied() {
+      return success;
+    }
+
+    /**
+     * Set directory SPS status failed.
+     */
+    public void setFailure(boolean failure) {
+      this.success = this.success || failure;
+    }
   }
 
   public void init() {
@@ -510,6 +532,11 @@ public class BlockStorageMovementNeeded {
       this.lastStatusUpdateTime = Time.monotonicNow();
     }
 
+    private void setFailure() {
+      this.status = StoragePolicySatisfyPathStatus.FAILURE;
+      this.lastStatusUpdateTime = Time.monotonicNow();
+    }
+
     private StoragePolicySatisfyPathStatus getStatus() {
       return status;
     }
@@ -518,7 +545,8 @@ public class BlockStorageMovementNeeded {
      * Return true if SUCCESS status cached more then 5 min.
      */
     private boolean canRemove() {
-      return StoragePolicySatisfyPathStatus.SUCCESS == status
+      return (StoragePolicySatisfyPathStatus.SUCCESS == status
+          || StoragePolicySatisfyPathStatus.FAILURE == status)
           && (Time.monotonicNow()
               - lastStatusUpdateTime) > statusClearanceElapsedTimeMs;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 7eb156c..897e61e 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
@@ -7693,6 +7693,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       final INode inode = dir.getInode(id);
+      if (inode == null) {
+        return;
+      }
       final XAttrFeature xaf = inode.getXAttrFeature();
       if (xaf == null) {
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 2382d36..972e744 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
@@ -83,6 +83,7 @@ public class StoragePolicySatisfier implements Runnable {
   private volatile boolean isRunning = false;
   private int spsWorkMultiplier;
   private long blockCount = 0L;
+  private int blockMovementMaxRetry;
   /**
    * Represents the collective analysis status for all blocks.
    */
@@ -137,6 +138,9 @@ public class StoragePolicySatisfier implements Runnable {
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
         storageMovementNeeded);
     this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(conf);
+    this.blockMovementMaxRetry = conf.getInt(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY,
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT);
   }
 
   /**
@@ -243,6 +247,13 @@ public class StoragePolicySatisfier implements Runnable {
         if (!namesystem.isInSafeMode()) {
           ItemInfo itemInfo = storageMovementNeeded.get();
           if (itemInfo != null) {
+            if(itemInfo.getRetryCount() >= blockMovementMaxRetry){
+              LOG.info("Failed to satisfy the policy after "
+                  + blockMovementMaxRetry + " retries. Removing inode "
+                  + itemInfo.getTrackId() + " from the queue");
+              storageMovementNeeded.removeItemTrackInfo(itemInfo, false);
+              continue;
+            }
             long trackId = itemInfo.getTrackId();
             BlockCollection blockCollection;
             BlocksMovingAnalysis status = null;
@@ -253,7 +264,7 @@ public class StoragePolicySatisfier implements Runnable {
               if (blockCollection == null) {
                 // File doesn't exists (maybe got deleted), remove trackId from
                 // the queue
-                storageMovementNeeded.removeItemTrackInfo(itemInfo);
+                storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
               } else {
                 status =
                     analyseBlocksStorageMovementsAndAssignToDN(
@@ -269,9 +280,9 @@ public class StoragePolicySatisfier implements Runnable {
                 // Just add to monitor, so it will be tracked for report and
                 // be removed on storage movement attempt finished report.
               case BLOCKS_TARGETS_PAIRED:
-                this.storageMovementsMonitor.add(new AttemptedItemInfo(
-                    itemInfo.getStartId(), itemInfo.getTrackId(),
-                    monotonicNow(), status.assignedBlocks));
+                this.storageMovementsMonitor.add(new AttemptedItemInfo(itemInfo
+                    .getStartId(), itemInfo.getTrackId(), monotonicNow(),
+                    status.assignedBlocks, itemInfo.getRetryCount()));
                 break;
               case NO_BLOCKS_TARGETS_PAIRED:
                 if (LOG.isDebugEnabled()) {
@@ -279,6 +290,7 @@ public class StoragePolicySatisfier implements Runnable {
                       + " back to retry queue as none of the blocks"
                       + " found its eligible targets.");
                 }
+                itemInfo.retryCount++;
                 this.storageMovementNeeded.add(itemInfo);
                 break;
               case FEW_LOW_REDUNDANCY_BLOCKS:
@@ -295,7 +307,7 @@ public class StoragePolicySatisfier implements Runnable {
               default:
                 LOG.info("Block analysis skipped or blocks already satisfied"
                     + " with storages. So, Cleaning up the Xattrs.");
-                storageMovementNeeded.removeItemTrackInfo(itemInfo);
+                storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
                 break;
               }
             }
@@ -861,10 +873,19 @@ public class StoragePolicySatisfier implements Runnable {
   public static class ItemInfo {
     private long startId;
     private long trackId;
+    private int retryCount;
 
     public ItemInfo(long startId, long trackId) {
       this.startId = startId;
       this.trackId = trackId;
+      //set 0 when item is getting added first time in queue.
+      this.retryCount = 0;
+    }
+
+    public ItemInfo(long startId, long trackId, int retryCount) {
+      this.startId = startId;
+      this.trackId = trackId;
+      this.retryCount = retryCount;
     }
 
     /**
@@ -887,6 +908,13 @@ public class StoragePolicySatisfier implements Runnable {
     public boolean isDir() {
       return (startId != trackId);
     }
+
+    /**
+     * Get the attempted retry count of the block for satisfy the policy.
+     */
+    public int getRetryCount() {
+      return retryCount;
+    }
   }
 
   /**
@@ -910,8 +938,8 @@ public class StoragePolicySatisfier implements Runnable {
      */
     AttemptedItemInfo(long rootId, long trackId,
         long lastAttemptedOrReportedTime,
-        List<Block> blocks) {
-      super(rootId, trackId);
+        List<Block> blocks, int retryCount) {
+      super(rootId, trackId, retryCount);
       this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
       this.blocks = blocks;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 05498d6..ee90eff 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
@@ -272,8 +272,11 @@ public class StoragePolicyAdmin extends Configured implements Tool {
               + " the policy in given path. This will print the current"
               + "status of the path in each 10 sec and status are:\n"
               + "PENDING : Path is in queue and not processed for satisfying"
-              + " the policy.\nIN_PROGRESS : Satisfying the storage policy for"
-              + " path.\nSUCCESS : Storage policy satisfied for the path.\n"
+              + " the policy.\n"
+              + "IN_PROGRESS : Satisfying the storage policy for"
+              + " path.\n"
+              + "SUCCESS : Storage policy satisfied for the path.\n"
+              + "FAILURE : Few blocks failed to move.\n"
               + "NOT_AVAILABLE : Status not available.");
       return getShortUsage() + "\n" +
           "Schedule blocks to move based on file/directory policy.\n\n" +
@@ -305,18 +308,30 @@ public class StoragePolicyAdmin extends Configured implements Tool {
       return 0;
     }
 
-
     private void waitForSatisfyPolicy(DistributedFileSystem dfs, String path)
         throws IOException {
       System.out.println("Waiting for satisfy the policy ...");
-      while (true) {
+      boolean running = true;
+      while (running) {
         StoragePolicySatisfyPathStatus status = dfs.getClient()
             .checkStoragePolicySatisfyPathStatus(path);
-        if (StoragePolicySatisfyPathStatus.SUCCESS.equals(status)) {
+        switch (status) {
+        case SUCCESS:
+        case FAILURE:
+        case NOT_AVAILABLE:
+          System.out.println(status);
+          running = false;
+          break;
+        case PENDING:
+        case IN_PROGRESS:
           System.out.println(status);
+        default:
+          System.err.println("Unexpected storage policy satisfyer status,"
+              + " Exiting");
+          running = false;
           break;
         }
-        System.out.println(status);
+
         try {
           Thread.sleep(10000);
         } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 16d9a5c..c971e7b 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
@@ -4405,12 +4405,21 @@
 
 <property>
   <name>dfs.storage.policy.satisfier.low.max-streams.preference</name>
-  <value>false</value>
+  <value>true</value>
   <description>
-    If true, blocks to move tasks will share equal ratio of number of highest-priority
+    If false, blocks to move tasks will share equal ratio of number of highest-priority
     replication streams (dfs.namenode.replication.max-streams) with pending replica and
-    erasure-coded reconstruction tasks. If false, blocks to move tasks will only use
-    the delta number of replication streams. The default value is false.
+    erasure-coded reconstruction tasks. If true, blocks to move tasks will only use
+    the delta number of replication streams. The default value is true.
+  </description>
+</property>
+
+<property>
+  <name>dfs.storage.policy.satisfier.retry.max.attempts</name>
+  <value>3</value>
+  <description>
+    Max retry to satisfy the block storage policy. After this retry block will be removed
+    from the movement needed queue.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 93fcb1b..eecb264 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -222,7 +222,7 @@ Schedule blocks to move based on file's/directory's current storage policy.
 | | |
 |:---- |:---- |
 | `-path <path>` | The path referring to either a directory or a file. |
-| `-w` | It requests that the command wait till all the files satisfy the policy in given path. This will print the current status of the path in each 10 sec and status are:<br/>PENDING - Path is in queue and not processed for satisfying the policy.<br/>IN_PROGRESS - Satisfying the storage policy for path.<br/>SUCCESS - Storage policy satisfied for the path.<br/>NOT_AVAILABLE - Status not available. |
+| `-w` | It requests that the command wait till all the files satisfy the policy in given path. This will print the current status of the path in each 10 sec and status are:<br/>PENDING - Path is in queue and not processed for satisfying the policy.<br/>IN_PROGRESS - Satisfying the storage policy for path.<br/>SUCCESS - Storage policy satisfied for the path.<br/>FAILURE : Few blocks failed to move.<br/>NOT_AVAILABLE - Status not available. |
 
 ### SPS Running Status
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 f79326f..d4ccb3e 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
@@ -91,7 +91,7 @@ public class TestBlockStorageMovementAttemptedItems {
     Long item = new Long(1234);
     List<Block> blocks = new ArrayList<Block>();
     blocks.add(new Block(item));
-    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks));
+    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks, 0));
     Block[] blockArray = new Block[blocks.size()];
     blocks.toArray(blockArray);
     bsmAttemptedItems.addReportedMovedBlocks(blockArray);
@@ -108,7 +108,7 @@ public class TestBlockStorageMovementAttemptedItems {
     Long item = new Long(1234);
     List<Block> blocks = new ArrayList<>();
     blocks.add(new Block(item));
-    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks));
+    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks, 0));
     assertEquals("Shouldn't receive result", 0,
         bsmAttemptedItems.getMovementFinishedBlocksCount());
     assertEquals("Item doesn't exist in the attempted list", 1,
@@ -129,7 +129,7 @@ public class TestBlockStorageMovementAttemptedItems {
     blocks.add(new Block(5678L));
     Long trackID = 0L;
     bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks));
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
     bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
@@ -154,7 +154,7 @@ public class TestBlockStorageMovementAttemptedItems {
     List<Block> blocks = new ArrayList<>();
     blocks.add(new Block(item));
     bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks));
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
     bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
@@ -182,7 +182,7 @@ public class TestBlockStorageMovementAttemptedItems {
     List<Block> blocks = new ArrayList<>();
     blocks.add(new Block(item));
     bsmAttemptedItems
-        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks));
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks, 0));
     Block[] blksMovementReport = new Block[1];
     blksMovementReport[0] = new Block(item);
     bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73f7db61/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 edd1aca..9f733ff 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
@@ -1412,8 +1412,8 @@ public class TestStoragePolicySatisfier {
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");
       config.setBoolean(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY,
-          true);
+          .DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_KEY,
+          false);
 
       StorageType[][] storagetypes = new StorageType[][] {
           {StorageType.ARCHIVE, StorageType.DISK},
@@ -1474,8 +1474,8 @@ public class TestStoragePolicySatisfier {
           .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
           "3000");
       config.setBoolean(DFSConfigKeys
-          .DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY,
-          true);
+          .DFS_STORAGE_POLICY_SATISFIER_LOW_MAX_STREAMS_PREFERENCE_KEY,
+          false);
 
       StorageType[][] storagetypes = new StorageType[][] {
           {StorageType.ARCHIVE, StorageType.DISK},
@@ -1531,6 +1531,55 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  @Test(timeout = 300000)
+  public void testMaxRetryForFailedBlock() throws Exception {
+    try {
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "1000");
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
+          "1000");
+      StorageType[][] storagetypes = new StorageType[][] {
+          {StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK}};
+      hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(2)
+          .storageTypes(storagetypes).build();
+      hdfsCluster.waitActive();
+      dfs = hdfsCluster.getFileSystem();
+
+      Path filePath = new Path("/retryFile");
+      DFSTestUtil.createFile(dfs, filePath, DEFAULT_BLOCK_SIZE, (short) 2,
+          0);
+
+      dfs.setStoragePolicy(filePath, "COLD");
+      dfs.satisfyStoragePolicy(filePath);
+      Thread.sleep(3000
+          * DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT);
+      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+          StorageType.DISK, 2, 60000, hdfsCluster.getFileSystem());
+      // Path status should be FAILURE
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            StoragePolicySatisfyPathStatus status = dfs.getClient()
+                .checkStoragePolicySatisfyPathStatus(filePath.toString());
+            return StoragePolicySatisfyPathStatus.FAILURE.equals(status);
+          } catch (IOException e) {
+            Assert.fail("Fail to get path status for sps");
+          }
+          return false;
+        }
+      }, 100, 90000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private static void createDirectoryTree(DistributedFileSystem dfs)
       throws Exception {
     // tree structure


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


[45/50] [abbrv] hadoop git commit: HDFS-12570: [SPS]: Refactor Co-ordinator datanode logic to track the block storage movements. Contributed by Rakesh R.

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 96c0fdf..a938fd3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -152,7 +152,7 @@ import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -1480,14 +1480,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
       @Nonnull SlowDiskReports slowDisks,
-      BlocksStorageMovementResult[] blkMovementStatus) throws IOException {
+      BlocksStorageMoveAttemptFinished storageMovementFinishedBlks)
+          throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
         dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
         failedVolumes, volumeFailureSummary, requestFullBlockReportLease,
         slowPeers, slowDisks,
-        blkMovementStatus);
+        storageMovementFinishedBlks);
   }
 
   @Override // DatanodeProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 a4372d5..a28a806 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.util.Time.monotonicNow;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -44,7 +46,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.Daemon;
@@ -82,25 +84,38 @@ public class StoragePolicySatisfier implements Runnable {
   /**
    * Represents the collective analysis status for all blocks.
    */
-  private enum BlocksMovingAnalysisStatus {
-    // Represents that, the analysis skipped due to some conditions. A such
-    // condition is if block collection is in incomplete state.
-    ANALYSIS_SKIPPED_FOR_RETRY,
-    // Represents that, all block storage movement needed blocks found its
-    // targets.
-    ALL_BLOCKS_TARGETS_PAIRED,
-    // Represents that, only fewer or none of the block storage movement needed
-    // block found its eligible targets.
-    FEW_BLOCKS_TARGETS_PAIRED,
-    // Represents that, none of the blocks found for block storage movements.
-    BLOCKS_ALREADY_SATISFIED,
-    // Represents that, the analysis skipped due to some conditions.
-    // Example conditions are if no blocks really exists in block collection or
-    // if analysis is not required on ec files with unsuitable storage policies
-    BLOCKS_TARGET_PAIRING_SKIPPED,
-    // Represents that, All the reported blocks are satisfied the policy but
-    // some of the blocks are low redundant.
-    FEW_LOW_REDUNDANCY_BLOCKS
+  private static class BlocksMovingAnalysis {
+
+    enum Status {
+      // Represents that, the analysis skipped due to some conditions. A such
+      // condition is if block collection is in incomplete state.
+      ANALYSIS_SKIPPED_FOR_RETRY,
+      // Represents that few or all blocks found respective target to do
+      // the storage movement.
+      BLOCKS_TARGETS_PAIRED,
+      // Represents that none of the blocks found respective target to do
+      // the storage movement.
+      NO_BLOCKS_TARGETS_PAIRED,
+      // Represents that, none of the blocks found for block storage movements.
+      BLOCKS_ALREADY_SATISFIED,
+      // Represents that, the analysis skipped due to some conditions.
+      // Example conditions are if no blocks really exists in block collection
+      // or
+      // if analysis is not required on ec files with unsuitable storage
+      // policies
+      BLOCKS_TARGET_PAIRING_SKIPPED,
+      // Represents that, All the reported blocks are satisfied the policy but
+      // some of the blocks are low redundant.
+      FEW_LOW_REDUNDANCY_BLOCKS
+    }
+
+    private Status status = null;
+    private List<Block> assignedBlocks = null;
+
+    BlocksMovingAnalysis(Status status, List<Block> blockMovingInfo) {
+      this.status = status;
+      this.assignedBlocks = blockMovingInfo;
+    }
   }
 
   public StoragePolicySatisfier(final Namesystem namesystem,
@@ -118,8 +133,7 @@ public class StoragePolicySatisfier implements Runnable {
         conf.getLong(
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
-        storageMovementNeeded,
-        this);
+        storageMovementNeeded);
     this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(conf);
   }
 
@@ -232,21 +246,25 @@ public class StoragePolicySatisfier implements Runnable {
                 namesystem.getBlockCollection(trackId);
             // Check blockCollectionId existence.
             if (blockCollection != null) {
-              BlocksMovingAnalysisStatus status =
+              BlocksMovingAnalysis status =
                   analyseBlocksStorageMovementsAndAssignToDN(blockCollection);
-              switch (status) {
+              switch (status.status) {
               // Just add to monitor, so it will be retried after timeout
               case ANALYSIS_SKIPPED_FOR_RETRY:
-                // Just add to monitor, so it will be tracked for result and
-                // be removed on successful storage movement result.
-              case ALL_BLOCKS_TARGETS_PAIRED:
-                this.storageMovementsMonitor.add(itemInfo, true);
+                // Just add to monitor, so it will be tracked for report and
+                // be removed on storage movement attempt finished report.
+              case BLOCKS_TARGETS_PAIRED:
+                this.storageMovementsMonitor.add(new AttemptedItemInfo(
+                    itemInfo.getStartId(), itemInfo.getTrackId(),
+                    monotonicNow(), status.assignedBlocks));
                 break;
-              // Add to monitor with allBlcoksAttemptedToSatisfy flag false, so
-              // that it will be tracked and still it will be consider for retry
-              // as analysis was not found targets for storage movement blocks.
-              case FEW_BLOCKS_TARGETS_PAIRED:
-                this.storageMovementsMonitor.add(itemInfo, false);
+              case NO_BLOCKS_TARGETS_PAIRED:
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Adding trackID " + trackId
+                      + " back to retry queue as none of the blocks"
+                      + " found its eligible targets.");
+                }
+                this.storageMovementNeeded.add(itemInfo);
                 break;
               case FEW_LOW_REDUNDANCY_BLOCKS:
                 if (LOG.isDebugEnabled()) {
@@ -310,10 +328,10 @@ public class StoragePolicySatisfier implements Runnable {
     return;
   }
 
-  private BlocksMovingAnalysisStatus analyseBlocksStorageMovementsAndAssignToDN(
+  private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
       BlockCollection blockCollection) {
-    BlocksMovingAnalysisStatus status =
-        BlocksMovingAnalysisStatus.BLOCKS_ALREADY_SATISFIED;
+    BlocksMovingAnalysis.Status status =
+        BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
     byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
     BlockStoragePolicy existingStoragePolicy =
         blockManager.getStoragePolicy(existingStoragePolicyID);
@@ -322,17 +340,18 @@ public class StoragePolicySatisfier implements Runnable {
       // So, should we add back? or leave it to user
       LOG.info("BlockCollectionID: {} file is under construction. So, postpone"
           + " this to the next retry iteration", blockCollection.getId());
-      return BlocksMovingAnalysisStatus.ANALYSIS_SKIPPED_FOR_RETRY;
+      return new BlocksMovingAnalysis(
+          BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
+          new ArrayList<>());
     }
 
-    // First datanode will be chosen as the co-ordinator node for storage
-    // movements. Later this can be optimized if needed.
-    DatanodeDescriptor coordinatorNode = null;
     BlockInfo[] blocks = blockCollection.getBlocks();
     if (blocks.length == 0) {
       LOG.info("BlockCollectionID: {} file is not having any blocks."
           + " So, skipping the analysis.", blockCollection.getId());
-      return BlocksMovingAnalysisStatus.BLOCKS_TARGET_PAIRING_SKIPPED;
+      return new BlocksMovingAnalysis(
+          BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
+          new ArrayList<>());
     }
     List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
 
@@ -352,7 +371,9 @@ public class StoragePolicySatisfier implements Runnable {
           LOG.warn("The storage policy " + existingStoragePolicy.getName()
               + " is not suitable for Striped EC files. "
               + "So, ignoring to move the blocks");
-          return BlocksMovingAnalysisStatus.BLOCKS_TARGET_PAIRING_SKIPPED;
+          return new BlocksMovingAnalysis(
+              BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
+              new ArrayList<>());
         }
       } else {
         expectedStorageTypes = existingStoragePolicy
@@ -370,30 +391,35 @@ public class StoragePolicySatisfier implements Runnable {
           new LinkedList<StorageType>(Arrays.asList(storageTypes));
       if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
           existing, true)) {
-        boolean computeStatus = computeBlockMovingInfos(blockMovingInfos,
+        boolean blocksPaired = computeBlockMovingInfos(blockMovingInfos,
             blockInfo, expectedStorageTypes, existing, storages);
-        if (computeStatus
-            && status != BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED
-            && !blockManager.hasLowRedundancyBlocks(blockCollection)) {
-          status = BlocksMovingAnalysisStatus.ALL_BLOCKS_TARGETS_PAIRED;
+        if (blocksPaired) {
+          status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
         } else {
-          status = BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED;
+          // none of the blocks found its eligible targets for satisfying the
+          // storage policy.
+          status = BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED;
         }
       } else {
         if (blockManager.hasLowRedundancyBlocks(blockCollection)) {
-          status = BlocksMovingAnalysisStatus.FEW_LOW_REDUNDANCY_BLOCKS;
+          status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
         }
       }
     }
 
-    assignBlockMovingInfosToCoordinatorDn(blockCollection.getId(),
-        blockMovingInfos, coordinatorNode);
-    int count = 0;
+    List<Block> assignedBlockIds = new ArrayList<Block>();
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      count = count + blkMovingInfo.getSources().length;
+      // Check for at least one block storage movement has been chosen
+      if (blkMovingInfo.getTarget() != null) {
+        // assign block storage movement task to the target node
+        ((DatanodeDescriptor) blkMovingInfo.getTarget())
+            .addBlocksToMoveStorage(blkMovingInfo);
+        LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
+        assignedBlockIds.add(blkMovingInfo.getBlock());
+        blockCount++;
+      }
     }
-    blockCount = blockCount + count;
-    return status;
+    return new BlocksMovingAnalysis(status, assignedBlockIds);
   }
 
   /**
@@ -468,41 +494,6 @@ public class StoragePolicySatisfier implements Runnable {
     return foundMatchingTargetNodesForBlock;
   }
 
-  private void assignBlockMovingInfosToCoordinatorDn(long blockCollectionID,
-      List<BlockMovingInfo> blockMovingInfos,
-      DatanodeDescriptor coordinatorNode) {
-
-    if (blockMovingInfos.size() < 1) {
-      // TODO: Major: handle this case. I think we need retry cases to
-      // be implemented. Idea is, if some files are not getting storage movement
-      // chances, then we can just retry limited number of times and exit.
-      return;
-    }
-
-    // For now, first datanode will be chosen as the co-ordinator. Later
-    // this can be optimized if needed.
-    coordinatorNode = (DatanodeDescriptor) blockMovingInfos.get(0)
-        .getSources()[0];
-
-    boolean needBlockStorageMovement = false;
-    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      // Check for atleast one block storage movement has been chosen
-      if (blkMovingInfo.getTargets().length > 0){
-        needBlockStorageMovement = true;
-        break;
-      }
-    }
-    if (!needBlockStorageMovement) {
-      // Simply return as there is no targets selected for scheduling the block
-      // movement.
-      return;
-    }
-
-    // 'BlockCollectionId' is used as the tracking ID. All the blocks under this
-    // blockCollectionID will be added to this datanode.
-    coordinatorNode.addBlocksToMoveStorage(blockCollectionID, blockMovingInfos);
-  }
-
   /**
    * Find the good target node for each source node for which block storages was
    * misplaced.
@@ -526,10 +517,6 @@ public class StoragePolicySatisfier implements Runnable {
       List<StorageType> expected,
       StorageTypeNodeMap locsForExpectedStorageTypes) {
     boolean foundMatchingTargetNodesForBlock = true;
-    List<DatanodeInfo> sourceNodes = new ArrayList<>();
-    List<StorageType> sourceStorageTypes = new ArrayList<>();
-    List<DatanodeInfo> targetNodes = new ArrayList<>();
-    List<StorageType> targetStorageTypes = new ArrayList<>();
     List<DatanodeDescriptor> excludeNodes = new ArrayList<>();
 
     // Looping over all the source node locations and choose the target
@@ -544,10 +531,15 @@ public class StoragePolicySatisfier implements Runnable {
         StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(
             blockInfo, existingTypeNodePair.dn, expected);
         if (chosenTarget != null) {
-          sourceNodes.add(existingTypeNodePair.dn);
-          sourceStorageTypes.add(existingTypeNodePair.storageType);
-          targetNodes.add(chosenTarget.dn);
-          targetStorageTypes.add(chosenTarget.storageType);
+          if (blockInfo.isStriped()) {
+            buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
+                existingTypeNodePair.storageType, chosenTarget.dn,
+                chosenTarget.storageType, blockMovingInfos);
+          } else {
+            buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
+                existingTypeNodePair.storageType, chosenTarget.dn,
+                chosenTarget.storageType, blockMovingInfos);
+          }
           expected.remove(chosenTarget.storageType);
           // TODO: We can increment scheduled block count for this node?
         }
@@ -563,7 +555,7 @@ public class StoragePolicySatisfier implements Runnable {
       StorageTypeNodePair chosenTarget = null;
       // Chosen the target storage within same datanode. So just skipping this
       // source node.
-      if (sourceNodes.contains(existingTypeNodePair.dn)) {
+      if (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
         continue;
       }
       if (chosenTarget == null && blockManager.getDatanodeManager()
@@ -586,10 +578,16 @@ public class StoragePolicySatisfier implements Runnable {
                 Matcher.ANY_OTHER, locsForExpectedStorageTypes, excludeNodes);
       }
       if (null != chosenTarget) {
-        sourceNodes.add(existingTypeNodePair.dn);
-        sourceStorageTypes.add(existingTypeNodePair.storageType);
-        targetNodes.add(chosenTarget.dn);
-        targetStorageTypes.add(chosenTarget.storageType);
+        if (blockInfo.isStriped()) {
+          buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
+              existingTypeNodePair.storageType, chosenTarget.dn,
+              chosenTarget.storageType, blockMovingInfos);
+        } else {
+          buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
+              existingTypeNodePair.storageType, chosenTarget.dn,
+              chosenTarget.storageType, blockMovingInfos);
+        }
+
         expected.remove(chosenTarget.storageType);
         excludeNodes.add(chosenTarget.dn);
         // TODO: We can increment scheduled block count for this node?
@@ -605,47 +603,33 @@ public class StoragePolicySatisfier implements Runnable {
       foundMatchingTargetNodesForBlock = false;
     }
 
-    blockMovingInfos.addAll(getBlockMovingInfos(blockInfo, sourceNodes,
-        sourceStorageTypes, targetNodes, targetStorageTypes));
     return foundMatchingTargetNodesForBlock;
   }
 
-  private List<BlockMovingInfo> getBlockMovingInfos(BlockInfo blockInfo,
-      List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
-      List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes) {
-    List<BlockMovingInfo> blkMovingInfos = new ArrayList<>();
-    // No source-target node pair exists.
-    if (sourceNodes.size() <= 0) {
-      return blkMovingInfos;
-    }
-
-    if (blockInfo.isStriped()) {
-      buildStripedBlockMovingInfos(blockInfo, sourceNodes, sourceStorageTypes,
-          targetNodes, targetStorageTypes, blkMovingInfos);
-    } else {
-      buildContinuousBlockMovingInfos(blockInfo, sourceNodes,
-          sourceStorageTypes, targetNodes, targetStorageTypes, blkMovingInfos);
+  private boolean checkIfAlreadyChosen(List<BlockMovingInfo> blockMovingInfos,
+      DatanodeDescriptor dn) {
+    for (BlockMovingInfo blockMovingInfo : blockMovingInfos) {
+      if (blockMovingInfo.getSource().equals(dn)) {
+        return true;
+      }
     }
-    return blkMovingInfos;
+    return false;
   }
 
   private void buildContinuousBlockMovingInfos(BlockInfo blockInfo,
-      List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
-      List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes,
+      DatanodeInfo sourceNode, StorageType sourceStorageType,
+      DatanodeInfo targetNode, StorageType targetStorageType,
       List<BlockMovingInfo> blkMovingInfos) {
     Block blk = new Block(blockInfo.getBlockId(), blockInfo.getNumBytes(),
         blockInfo.getGenerationStamp());
-    BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk,
-        sourceNodes.toArray(new DatanodeInfo[sourceNodes.size()]),
-        targetNodes.toArray(new DatanodeInfo[targetNodes.size()]),
-        sourceStorageTypes.toArray(new StorageType[sourceStorageTypes.size()]),
-        targetStorageTypes.toArray(new StorageType[targetStorageTypes.size()]));
+    BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
+        targetNode, sourceStorageType, targetStorageType);
     blkMovingInfos.add(blkMovingInfo);
   }
 
   private void buildStripedBlockMovingInfos(BlockInfo blockInfo,
-      List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
-      List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes,
+      DatanodeInfo sourceNode, StorageType sourceStorageType,
+      DatanodeInfo targetNode, StorageType targetStorageType,
       List<BlockMovingInfo> blkMovingInfos) {
     // For a striped block, it needs to construct internal block at the given
     // index of a block group. Here it is iterating over all the block indices
@@ -655,30 +639,17 @@ public class StoragePolicySatisfier implements Runnable {
     for (StorageAndBlockIndex si : sBlockInfo.getStorageAndIndexInfos()) {
       if (si.getBlockIndex() >= 0) {
         DatanodeDescriptor dn = si.getStorage().getDatanodeDescriptor();
-        DatanodeInfo[] srcNode = new DatanodeInfo[1];
-        StorageType[] srcStorageType = new StorageType[1];
-        DatanodeInfo[] targetNode = new DatanodeInfo[1];
-        StorageType[] targetStorageType = new StorageType[1];
-        for (int i = 0; i < sourceNodes.size(); i++) {
-          DatanodeInfo node = sourceNodes.get(i);
-          if (node.equals(dn)) {
-            srcNode[0] = node;
-            srcStorageType[0] = sourceStorageTypes.get(i);
-            targetNode[0] = targetNodes.get(i);
-            targetStorageType[0] = targetStorageTypes.get(i);
-
-            // construct internal block
-            long blockId = blockInfo.getBlockId() + si.getBlockIndex();
-            long numBytes = StripedBlockUtil.getInternalBlockLength(
-                sBlockInfo.getNumBytes(), sBlockInfo.getCellSize(),
-                sBlockInfo.getDataBlockNum(), si.getBlockIndex());
-            Block blk = new Block(blockId, numBytes,
-                blockInfo.getGenerationStamp());
-            BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, srcNode,
-                targetNode, srcStorageType, targetStorageType);
-            blkMovingInfos.add(blkMovingInfo);
-            break; // found matching source-target nodes
-          }
+        if (sourceNode.equals(dn)) {
+          // construct internal block
+          long blockId = blockInfo.getBlockId() + si.getBlockIndex();
+          long numBytes = StripedBlockUtil.getInternalBlockLength(
+              sBlockInfo.getNumBytes(), sBlockInfo.getCellSize(),
+              sBlockInfo.getDataBlockNum(), si.getBlockIndex());
+          Block blk = new Block(blockId, numBytes,
+              blockInfo.getGenerationStamp());
+          BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
+              targetNode, sourceStorageType, targetStorageType);
+          blkMovingInfos.add(blkMovingInfo);
         }
       }
     }
@@ -817,18 +788,18 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   /**
-   * Receives the movement results of collection of blocks associated to a
-   * trackId.
+   * Receives set of storage movement attempt finished blocks report.
    *
-   * @param blksMovementResults
-   *          movement status of the set of blocks associated to a trackId.
+   * @param moveAttemptFinishedBlks
+   *          set of storage movement attempt finished blocks.
    */
-  void handleBlocksStorageMovementResults(
-      BlocksStorageMovementResult[] blksMovementResults) {
-    if (blksMovementResults.length <= 0) {
+  void handleStorageMovementAttemptFinishedBlks(
+      BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks) {
+    if (moveAttemptFinishedBlks.getBlocks().length <= 0) {
       return;
     }
-    storageMovementsMonitor.addResults(blksMovementResults);
+    storageMovementsMonitor
+        .addReportedMovedBlocks(moveAttemptFinishedBlks.getBlocks());
   }
 
   @VisibleForTesting
@@ -906,4 +877,52 @@ public class StoragePolicySatisfier implements Runnable {
       return (startId != trackId);
     }
   }
+
+  /**
+   * This class contains information of an attempted blocks and its last
+   * attempted or reported time stamp. This is used by
+   * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
+   */
+  final static class AttemptedItemInfo extends ItemInfo {
+    private long lastAttemptedOrReportedTime;
+    private final List<Block> blocks;
+
+    /**
+     * AttemptedItemInfo constructor.
+     *
+     * @param rootId
+     *          rootId for trackId
+     * @param trackId
+     *          trackId for file.
+     * @param lastAttemptedOrReportedTime
+     *          last attempted or reported time
+     */
+    AttemptedItemInfo(long rootId, long trackId,
+        long lastAttemptedOrReportedTime,
+        List<Block> blocks) {
+      super(rootId, trackId);
+      this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
+      this.blocks = blocks;
+    }
+
+    /**
+     * @return last attempted or reported time stamp.
+     */
+    long getLastAttemptedOrReportedTime() {
+      return lastAttemptedOrReportedTime;
+    }
+
+    /**
+     * Update lastAttemptedOrReportedTime, so that the expiration time will be
+     * postponed to future.
+     */
+    void touchLastReportedTimeStamp() {
+      this.lastAttemptedOrReportedTime = monotonicNow();
+    }
+
+    List<Block> getBlocks() {
+      return this.blocks;
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
index 5dcf4e7..e90317d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.hadoop.fs.StorageType;
@@ -29,22 +28,15 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
  * given set of blocks to specified target DataNodes to fulfill the block
  * storage policy.
  *
- * Upon receiving this command, this DataNode coordinates all the block movement
- * by passing the details to
+ * Upon receiving this command, this DataNode pass the array of block movement
+ * details to
  * {@link org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker}
- * service. After the block movement this DataNode sends response back to the
- * NameNode about the movement status.
- *
- * The coordinator datanode will use 'trackId' identifier to coordinate the
- * block movement of the given set of blocks. TrackId is a unique identifier
- * that represents a group of blocks. Namenode will generate this unique value
- * and send it to the coordinator datanode along with the
- * BlockStorageMovementCommand. Datanode will monitor the completion of the
- * block movements that grouped under this trackId and notifies Namenode about
- * the completion status.
+ * service. Later, StoragePolicySatisfyWorker will schedule block movement tasks
+ * for these blocks and monitors the completion of each task. After the block
+ * movement attempt is finished(with success or failure) this DataNode will send
+ * response back to NameNode about the block movement attempt finished details.
  */
 public class BlockStorageMovementCommand extends DatanodeCommand {
-  private final long trackID;
   private final String blockPoolId;
   private final Collection<BlockMovingInfo> blockMovingTasks;
 
@@ -53,30 +45,17 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
    *
    * @param action
    *          protocol specific action
-   * @param trackID
-   *          unique identifier to monitor the given set of block movements
-   * @param blockPoolId
-   *          block pool ID
    * @param blockMovingInfos
    *          block to storage info that will be used for movement
    */
-  public BlockStorageMovementCommand(int action, long trackID,
-      String blockPoolId, Collection<BlockMovingInfo> blockMovingInfos) {
+  public BlockStorageMovementCommand(int action, String blockPoolId,
+      Collection<BlockMovingInfo> blockMovingInfos) {
     super(action);
-    this.trackID = trackID;
     this.blockPoolId = blockPoolId;
     this.blockMovingTasks = blockMovingInfos;
   }
 
   /**
-   * Returns trackID, which will be used to monitor the block movement assigned
-   * to this coordinator datanode.
-   */
-  public long getTrackID() {
-    return trackID;
-  }
-
-  /**
    * Returns block pool ID.
    */
   public String getBlockPoolId() {
@@ -95,33 +74,29 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
    */
   public static class BlockMovingInfo {
     private Block blk;
-    private DatanodeInfo[] sourceNodes;
-    private DatanodeInfo[] targetNodes;
-    private StorageType[] sourceStorageTypes;
-    private StorageType[] targetStorageTypes;
+    private DatanodeInfo sourceNode;
+    private DatanodeInfo targetNode;
+    private StorageType sourceStorageType;
+    private StorageType targetStorageType;
 
     /**
      * Block to storage info constructor.
      *
      * @param block
-     *          block
-     * @param sourceDnInfos
-     *          node that can be the sources of a block move
-     * @param targetDnInfos
-     *          target datanode info
-     * @param srcStorageTypes
+     *          block info
+     * @param sourceDnInfo
+     *          node that can be the source of a block move
+     * @param srcStorageType
      *          type of source storage media
-     * @param targetStorageTypes
-     *          type of destin storage media
      */
-    public BlockMovingInfo(Block block,
-        DatanodeInfo[] sourceDnInfos, DatanodeInfo[] targetDnInfos,
-        StorageType[] srcStorageTypes, StorageType[] targetStorageTypes) {
+    public BlockMovingInfo(Block block, DatanodeInfo sourceDnInfo,
+        DatanodeInfo targetDnInfo, StorageType srcStorageType,
+        StorageType targetStorageType) {
       this.blk = block;
-      this.sourceNodes = sourceDnInfos;
-      this.targetNodes = targetDnInfos;
-      this.sourceStorageTypes = srcStorageTypes;
-      this.targetStorageTypes = targetStorageTypes;
+      this.sourceNode = sourceDnInfo;
+      this.targetNode = targetDnInfo;
+      this.sourceStorageType = srcStorageType;
+      this.targetStorageType = targetStorageType;
     }
 
     public void addBlock(Block block) {
@@ -129,35 +104,33 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
     }
 
     public Block getBlock() {
-      return this.blk;
+      return blk;
     }
 
-    public DatanodeInfo[] getSources() {
-      return sourceNodes;
+    public DatanodeInfo getSource() {
+      return sourceNode;
     }
 
-    public DatanodeInfo[] getTargets() {
-      return targetNodes;
+    public DatanodeInfo getTarget() {
+      return targetNode;
     }
 
-    public StorageType[] getTargetStorageTypes() {
-      return targetStorageTypes;
+    public StorageType getTargetStorageType() {
+      return targetStorageType;
     }
 
-    public StorageType[] getSourceStorageTypes() {
-      return sourceStorageTypes;
+    public StorageType getSourceStorageType() {
+      return sourceStorageType;
     }
 
     @Override
     public String toString() {
       return new StringBuilder().append("BlockMovingInfo(\n  ")
           .append("Moving block: ").append(blk).append(" From: ")
-          .append(Arrays.asList(sourceNodes)).append(" To: [")
-          .append(Arrays.asList(targetNodes)).append("\n  ")
-          .append(" sourceStorageTypes: ")
-          .append(Arrays.toString(sourceStorageTypes))
-          .append(" targetStorageTypes: ")
-          .append(Arrays.toString(targetStorageTypes)).append(")").toString();
+          .append(sourceNode).append(" To: [").append(targetNode).append("\n  ")
+          .append(" sourceStorageType: ").append(sourceStorageType)
+          .append(" targetStorageType: ").append(targetStorageType).append(")")
+          .toString();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMoveAttemptFinished.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMoveAttemptFinished.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMoveAttemptFinished.java
new file mode 100644
index 0000000..c837e013
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMoveAttemptFinished.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * This class represents, the blocks for which storage movements has done by
+ * datanodes. The movementFinishedBlocks array contains all the blocks that are
+ * attempted to do the movement and it could be finished with either success or
+ * failure.
+ */
+public class BlocksStorageMoveAttemptFinished {
+
+  private final Block[] movementFinishedBlocks;
+
+  public BlocksStorageMoveAttemptFinished(Block[] moveAttemptFinishedBlocks) {
+    this.movementFinishedBlocks = moveAttemptFinishedBlocks;
+  }
+
+  public Block[] getBlocks() {
+    return movementFinishedBlocks;
+  }
+
+  @Override
+  public String toString() {
+    return new StringBuilder().append("BlocksStorageMovementFinished(\n  ")
+        .append("  blockID: ").append(Arrays.toString(movementFinishedBlocks))
+        .append(")").toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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
deleted file mode 100644
index 7f749ec4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMovementResult.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.protocol;
-
-/**
- * This class represents, movement status of a set of blocks associated to a
- * track Id.
- */
-public class BlocksStorageMovementResult {
-
-  private final long trackId;
-  private final Status status;
-
-  /**
-   * SUCCESS - If all the blocks associated to track id has moved successfully
-   * or maximum possible movements done.
-   *
-   * <p>
-   * FAILURE - If any of its(trackId) blocks movement failed and requires to
-   * retry these failed blocks movements. Example selected target node is no
-   * more running or no space. So, retrying by selecting new target node might
-   * work.
-   *
-   * <p>
-   * IN_PROGRESS - If all or some of the blocks associated to track id are
-   * still moving.
-   */
-  public enum Status {
-    SUCCESS, FAILURE, IN_PROGRESS;
-  }
-
-  /**
-   * BlocksStorageMovementResult constructor.
-   *
-   * @param trackId
-   *          tracking identifier
-   * @param status
-   *          block movement status
-   */
-  public BlocksStorageMovementResult(long trackId, Status status) {
-    this.trackId = trackId;
-    this.status = status;
-  }
-
-  public long getTrackId() {
-    return trackId;
-  }
-
-  public Status getStatus() {
-    return status;
-  }
-
-  @Override
-  public String toString() {
-    return new StringBuilder().append("BlocksStorageMovementResult(\n  ")
-        .append("track id: ").append(trackId).append("  status: ")
-        .append(status).append(")").toString();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 5e1f148..fcc2df1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -112,8 +112,7 @@ public interface DatanodeProtocol {
    * @param slowPeers Details of peer DataNodes that were detected as being
    *                  slow to respond to packet writes. Empty report if no
    *                  slow peers were detected by the DataNode.
-   * @param blksMovementResults array of movement status of a set of blocks
-   *                            associated to a trackId.
+   * @param storageMovFinishedBlks array of movement attempt finished blocks
    * @throws IOException on error
    */
   @Idempotent
@@ -128,7 +127,7 @@ public interface DatanodeProtocol {
                                        boolean requestFullBlockReportLease,
                                        @Nonnull SlowPeerReports slowPeers,
                                        @Nonnull SlowDiskReports slowDisks,
-                                       BlocksStorageMovementResult[] blksMovementResults)
+                                       BlocksStorageMoveAttemptFinished storageMovFinishedBlks)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 080f7fa..7c35494 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -162,9 +162,8 @@ message BlockECReconstructionCommandProto {
  * Block storage movement command
  */
 message BlockStorageMovementCommandProto {
-  required uint64 trackID = 1;
-  required string blockPoolId = 2;
-  repeated BlockStorageMovementProto blockStorageMovement = 3;
+  required string blockPoolId = 1;
+  repeated BlockMovingInfoProto blockMovingInfo = 2;
 }
 
 /**
@@ -177,25 +176,20 @@ message DropSPSWorkCommandProto {
 /**
  * Block storage movement information
  */
-message BlockStorageMovementProto {
+message BlockMovingInfoProto {
   required BlockProto block = 1;
-  required DatanodeInfosProto sourceDnInfos = 2;
-  required DatanodeInfosProto targetDnInfos = 3;
-  required StorageTypesProto sourceStorageTypes = 4;
-  required StorageTypesProto targetStorageTypes = 5;
+  required DatanodeInfoProto sourceDnInfo = 2;
+  required DatanodeInfoProto targetDnInfo = 3;
+  required StorageTypeProto sourceStorageType = 4;
+  required StorageTypeProto targetStorageType = 5;
 }
 
 /**
- * Movement status of the set of blocks associated to a trackId.
+ * Blocks for which storage movements has been attempted and finished
+ * with either success or failure.
  */
-message BlocksStorageMovementResultProto {
-  enum Status {
-    SUCCESS = 1; // block movement succeeded
-    FAILURE = 2; // block movement failed and needs to retry
-    IN_PROGRESS = 3; // block movement is still in progress
-  }
-  required uint64 trackID = 1;
-  required Status status = 2;
+message BlocksStorageMoveAttemptFinishedProto {
+  repeated BlockProto blocks = 1;
 }
 
 /**
@@ -255,7 +249,7 @@ message HeartbeatRequestProto {
   optional bool requestFullBlockReportLease = 9 [ default = false ];
   repeated SlowPeerReportProto slowPeers = 10;
   repeated SlowDiskReportProto slowDisks = 11;
-  repeated BlocksStorageMovementResultProto blksMovementResults = 12;
+  optional BlocksStorageMoveAttemptFinishedProto storageMoveAttemptFinishedBlks = 12;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 ef0bf12..16d9a5c 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
@@ -4382,24 +4382,35 @@
 
 <property>
   <name>dfs.storage.policy.satisfier.recheck.timeout.millis</name>
-  <value>300000</value>
+  <value>60000</value>
   <description>
     Blocks storage movements monitor re-check interval in milliseconds.
     This check will verify whether any blocks storage movement results arrived from DN
     and also verify if any of file blocks movements not at all reported to DN
     since dfs.storage.policy.satisfier.self.retry.timeout.
-    The default value is 5 * 60 * 1000 (5 mins)
+    The default value is 1 * 60 * 1000 (1 mins)
   </description>
 </property>
 
 <property>
   <name>dfs.storage.policy.satisfier.self.retry.timeout.millis</name>
-  <value>1800000</value>
+  <value>300000</value>
   <description>
-    If any of file related block movements not at all reported by coordinator datanode,
+    If any of file related block movements not at all reported by datanode,
     then after this timeout(in milliseconds), the item will be added back to movement needed list
     at namenode which will be retried for block movements.
-    The default value is 30 * 60 * 1000 (30 mins)
+    The default value is 5 * 60 * 1000 (5 mins)
+  </description>
+</property>
+
+<property>
+  <name>dfs.storage.policy.satisfier.low.max-streams.preference</name>
+  <value>false</value>
+  <description>
+    If true, blocks to move tasks will share equal ratio of number of highest-priority
+    replication streams (dfs.namenode.replication.max-streams) with pending replica and
+    erasure-coded reconstruction tasks. If false, blocks to move tasks will only use
+    the delta number of replication streams. The default value is false.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 da61842..f69a3ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -104,7 +104,7 @@ Following 2 options will allow users to move the blocks based on new policy set.
 ### <u>S</u>torage <u>P</u>olicy <u>S</u>atisfier (SPS)
 
 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.
+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. If there are any failures in movement, the SPS will re-attempt by sending new block movement tasks.
 
 SPS can be enabled and disabled dynamically without restarting the Namenode.
 
@@ -127,10 +127,10 @@ Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HD
    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.
+   command results from Datanodes.
 
 *   **dfs.storage.policy.satisfier.self.retry.timeout.millis** - A timeout to retry if no block movement results reported from
-   Co-ordinator Datanode in this configured timeout.
+   Datanode in this configured timeout.
 
 ### Mover - A New Data Migration Tool
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 2d58732..fc6214e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
@@ -117,7 +117,8 @@ public class TestNameNodePrunesMissingStorages {
       cluster.stopDataNode(0);
       cluster.getNameNodeRpc().sendHeartbeat(dnReg, prunedReports, 0L, 0L, 0, 0,
           0, null, true, SlowPeerReports.EMPTY_REPORT,
-          SlowDiskReports.EMPTY_REPORT, new BlocksStorageMovementResult[0]);
+          SlowDiskReports.EMPTY_REPORT,
+          new BlocksStorageMoveAttemptFinished(null));
 
       // Check that the missing storage was pruned.
       assertThat(dnDescriptor.getStorageInfos().length, is(expectedStoragesAfterTest));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
index bd831d6..d13d717 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
@@ -169,7 +169,7 @@ public class InternalDataNodeTestUtils {
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
             Mockito.any(SlowDiskReports.class),
-            Mockito.any(BlocksStorageMovementResult[].class))).thenReturn(
+            Mockito.any(BlocksStorageMoveAttemptFinished.class))).thenReturn(
         new HeartbeatResponse(new DatanodeCommand[0], new NNHAStatusHeartbeat(
             HAServiceState.ACTIVE, 1), null, ThreadLocalRandom.current()
             .nextLong() | 1L));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index 524b8b9..4c75ff8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -160,7 +160,7 @@ public class TestBPOfferService {
           Mockito.anyBoolean(),
           Mockito.any(SlowPeerReports.class),
           Mockito.any(SlowDiskReports.class),
-          Mockito.any(BlocksStorageMovementResult[].class));
+          Mockito.any(BlocksStorageMoveAttemptFinished.class));
     mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     datanodeCommands[nnIdx] = new DatanodeCommand[0];
     return mock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 38372ba..2327237 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -87,7 +87,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -228,7 +228,7 @@ public class TestBlockRecovery {
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
             Mockito.any(SlowDiskReports.class),
-            Mockito.any(BlocksStorageMovementResult[].class)))
+            Mockito.any(BlocksStorageMoveAttemptFinished.class)))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
             new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
index b15b530..0dd15c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
@@ -174,7 +174,7 @@ public class TestDataNodeLifeline {
             anyBoolean(),
             any(SlowPeerReports.class),
             any(SlowDiskReports.class),
-            any(BlocksStorageMovementResult[].class));
+            any(BlocksStorageMoveAttemptFinished.class));
 
     // Intercept lifeline to trigger latch count-down on each call.
     doAnswer(new LatchCountingAnswer<Void>(lifelinesSent))
@@ -240,7 +240,7 @@ public class TestDataNodeLifeline {
             anyBoolean(),
             any(SlowPeerReports.class),
             any(SlowDiskReports.class),
-            any(BlocksStorageMovementResult[].class));
+            any(BlocksStorageMoveAttemptFinished.class));
 
     // While waiting on the latch for the expected number of heartbeat messages,
     // poll DataNode tracking information.  We expect that the DataNode always

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
index d7ac3f9..d47da69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
@@ -224,7 +224,7 @@ public class TestDatanodeProtocolRetryPolicy {
            Mockito.anyBoolean(),
            Mockito.any(SlowPeerReports.class),
            Mockito.any(SlowDiskReports.class),
-           Mockito.any(BlocksStorageMovementResult[].class));
+           Mockito.any(BlocksStorageMoveAttemptFinished.class));
 
     dn = new DataNode(conf, locations, null, null) {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
index b9f21a0..3732b2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
@@ -66,7 +66,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.Page
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -210,7 +210,7 @@ public class TestFsDatasetCache {
           anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
           anyBoolean(), any(SlowPeerReports.class),
           any(SlowDiskReports.class),
-          (BlocksStorageMovementResult[]) any());
+          any(BlocksStorageMoveAttemptFinished.class));
     } finally {
       lock.writeLock().unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 b84b1d2..3681cae 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
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -36,8 +36,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -180,11 +178,10 @@ public class TestStoragePolicySatisfyWorker {
           lb.getBlock().getLocalBlock(), lb.getLocations()[0], targetDnInfo,
           lb.getStorageTypes()[0], StorageType.ARCHIVE);
       blockMovingInfos.add(blockMovingInfo);
-      INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
-      worker.processBlockMovingTasks(inode.getId(),
-          cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
+      worker.processBlockMovingTasks(cluster.getNamesystem().getBlockPoolId(),
+          blockMovingInfos);
 
-      waitForBlockMovementCompletion(worker, inode.getId(), 1, 30000);
+      waitForBlockMovementCompletion(worker, 1, 30000);
     } finally {
       worker.stop();
     }
@@ -226,50 +223,42 @@ public class TestStoragePolicySatisfyWorker {
                 locatedBlock.getStorageTypes()[0], StorageType.ARCHIVE);
         blockMovingInfos.add(blockMovingInfo);
       }
-      INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
-      worker.processBlockMovingTasks(inode.getId(),
-          cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
+      worker.processBlockMovingTasks(cluster.getNamesystem().getBlockPoolId(),
+          blockMovingInfos);
       // Wait till results queue build up
-      waitForBlockMovementResult(worker, inode.getId(), 30000);
+      waitForBlockMovementResult(worker, 30000);
       worker.dropSPSWork();
       assertTrue(worker.getBlocksMovementsStatusHandler()
-          .getBlksMovementResults().size() == 0);
+          .getMoveAttemptFinishedBlocks().size() == 0);
     } finally {
       worker.stop();
     }
   }
 
   private void waitForBlockMovementResult(
-      final StoragePolicySatisfyWorker worker, final long inodeId, int timeout)
-          throws Exception {
+      final StoragePolicySatisfyWorker worker, int timeout) throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        List<BlocksStorageMovementResult> completedBlocks = worker
-            .getBlocksMovementsStatusHandler().getBlksMovementResults();
+        List<Block> completedBlocks = worker.getBlocksMovementsStatusHandler()
+            .getMoveAttemptFinishedBlocks();
         return completedBlocks.size() > 0;
       }
     }, 100, timeout);
   }
 
   private void waitForBlockMovementCompletion(
-      final StoragePolicySatisfyWorker worker, final long inodeId,
-      int expectedFailedItemsCount, int timeout) throws Exception {
+      final StoragePolicySatisfyWorker worker,
+      int expectedFinishedItemsCount, int timeout) throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        List<BlocksStorageMovementResult> completedBlocks = worker
-            .getBlocksMovementsStatusHandler().getBlksMovementResults();
-        int failedCount = 0;
-        for (BlocksStorageMovementResult blkMovementResult : completedBlocks) {
-          if (blkMovementResult.getStatus() ==
-              BlocksStorageMovementResult.Status.FAILURE) {
-            failedCount++;
-          }
-        }
+        List<Block> completedBlocks = worker.getBlocksMovementsStatusHandler()
+            .getMoveAttemptFinishedBlocks();
+        int finishedCount = completedBlocks.size();
         LOG.info("Block movement completed count={}, expected={} and actual={}",
-            completedBlocks.size(), expectedFailedItemsCount, failedCount);
-        return expectedFailedItemsCount == failedCount;
+            completedBlocks.size(), expectedFinishedItemsCount, finishedCount);
+        return expectedFinishedItemsCount == finishedCount;
       }
     }, 100, timeout);
   }
@@ -304,8 +293,7 @@ public class TestStoragePolicySatisfyWorker {
   private BlockMovingInfo prepareBlockMovingInfo(Block block,
       DatanodeInfo src, DatanodeInfo destin, StorageType storageType,
       StorageType targetStorageType) {
-    return new BlockMovingInfo(block, new DatanodeInfo[] {src},
-        new DatanodeInfo[] {destin}, new StorageType[] {storageType},
-        new StorageType[] {targetStorageType});
+    return new BlockMovingInfo(block, src, destin, storageType,
+        targetStorageType);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
index df120ca..20402f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
@@ -112,7 +112,7 @@ public class TestStorageReport {
         Mockito.any(VolumeFailureSummary.class), Mockito.anyBoolean(),
         Mockito.any(SlowPeerReports.class),
         Mockito.any(SlowDiskReports.class),
-        Mockito.any(BlocksStorageMovementResult[].class));
+        Mockito.any(BlocksStorageMoveAttemptFinished.class));
 
     StorageReport[] reports = captor.getValue();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 1e016f7..ec00ae7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -958,7 +958,7 @@ public class NNThroughputBenchmark implements Tool {
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration, rep,
           0L, 0L, 0, 0, 0, null, true,
           SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
-          new BlocksStorageMovementResult[0]).getCommands();
+          new BlocksStorageMoveAttemptFinished(null)).getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -1009,7 +1009,7 @@ public class NNThroughputBenchmark implements Tool {
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration,
           rep, 0L, 0L, 0, 0, 0, null, true,
           SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
-          new BlocksStorageMovementResult[0]).getCommands();
+          new BlocksStorageMoveAttemptFinished(null)).getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index ba29c82..b2b878d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
@@ -128,7 +128,7 @@ public class NameNodeAdapter {
         BlockManagerTestUtil.getStorageReportsForDatanode(dd),
         dd.getCacheCapacity(), dd.getCacheRemaining(), 0, 0, 0, null, true,
         SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
-        new BlocksStorageMovementResult[0]);
+        new BlocksStorageMoveAttemptFinished(null));
   }
 
   public static boolean setReplication(final FSNamesystem ns,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 7918821..f79326f 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
@@ -18,10 +18,17 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.util.Time.monotonicNow;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.AttemptedItemInfo;
 
 import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,9 +49,8 @@ public class TestBlockStorageMovementAttemptedItems {
     unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(
         Mockito.mock(Namesystem.class),
         Mockito.mock(StoragePolicySatisfier.class), 100);
-    StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
     bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
-        selfRetryTimeout, unsatisfiedStorageMovementFiles, sps);
+        selfRetryTimeout, unsatisfiedStorageMovementFiles);
   }
 
   @After
@@ -76,120 +82,115 @@ public class TestBlockStorageMovementAttemptedItems {
     return isItemFound;
   }
 
+  /**
+   * Verify that moved blocks reporting should queued up the block info.
+   */
   @Test(timeout = 30000)
-  public void testAddResultWithFailureResult() throws Exception {
-    bsmAttemptedItems.start(); // start block movement result monitor thread
-    Long item = new Long(1234);
-    bsmAttemptedItems.add(new ItemInfo(0L, item), true);
-    bsmAttemptedItems.addResults(
-        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
-            item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
-    assertTrue(checkItemMovedForRetry(item, 200));
-  }
-
-  @Test(timeout = 30000)
-  public void testAddResultWithSucessResult() throws Exception {
+  public void testAddReportedMoveAttemptFinishedBlocks() throws Exception {
     bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(new ItemInfo(0L, item), true);
-    bsmAttemptedItems.addResults(
-        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
-            item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
-    assertFalse(checkItemMovedForRetry(item, 200));
+    List<Block> blocks = new ArrayList<Block>();
+    blocks.add(new Block(item));
+    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks));
+    Block[] blockArray = new Block[blocks.size()];
+    blocks.toArray(blockArray);
+    bsmAttemptedItems.addReportedMovedBlocks(blockArray);
+    assertEquals("Failed to receive result!", 1,
+        bsmAttemptedItems.getMovementFinishedBlocksCount());
   }
 
+  /**
+   * Verify empty moved blocks reporting queue.
+   */
   @Test(timeout = 30000)
-  public void testNoResultAdded() throws Exception {
-    bsmAttemptedItems.start(); // start block movement result monitor thread
+  public void testNoBlockMovementAttemptFinishedReportAdded() throws Exception {
+    bsmAttemptedItems.start(); // start block movement report monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(new ItemInfo(0L, item), true);
-    // After self retry timeout, it should be added back for retry
-    assertTrue("Failed to add to the retry list",
-        checkItemMovedForRetry(item, 600));
-    assertEquals("Failed to remove from the attempted list", 0,
+    List<Block> blocks = new ArrayList<>();
+    blocks.add(new Block(item));
+    bsmAttemptedItems.add(new AttemptedItemInfo(0L, 0L, 0L, blocks));
+    assertEquals("Shouldn't receive result", 0,
+        bsmAttemptedItems.getMovementFinishedBlocksCount());
+    assertEquals("Item doesn't exist in the attempted list", 1,
         bsmAttemptedItems.getAttemptedItemsCount());
   }
 
   /**
-   * Partial block movement with BlocksStorageMovementResult#SUCCESS. Here,
-   * first occurrence is #blockStorageMovementResultCheck() and then
+   * Partial block movement with
+   * BlockMovementStatus#DN_BLK_STORAGE_MOVEMENT_SUCCESS. Here, first occurrence
+   * is #blockStorageMovementReportedItemsCheck() and then
    * #blocksStorageMovementUnReportedItemsCheck().
    */
   @Test(timeout = 30000)
   public void testPartialBlockMovementShouldBeRetried1() throws Exception {
     Long item = new Long(1234);
-    bsmAttemptedItems.add(new ItemInfo(0L, item), false);
-    bsmAttemptedItems.addResults(
-        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
-            item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
-
-    // start block movement result monitor thread
+    List<Block> blocks = new ArrayList<>();
+    blocks.add(new Block(item));
+    blocks.add(new Block(5678L));
+    Long trackID = 0L;
+    bsmAttemptedItems
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks));
+    Block[] blksMovementReport = new Block[1];
+    blksMovementReport[0] = new Block(item);
+    bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
+
+    // start block movement report monitor thread
     bsmAttemptedItems.start();
     assertTrue("Failed to add to the retry list",
-        checkItemMovedForRetry(item, 5000));
+        checkItemMovedForRetry(trackID, 5000));
     assertEquals("Failed to remove from the attempted list", 0,
         bsmAttemptedItems.getAttemptedItemsCount());
   }
 
   /**
-   * Partial block movement with BlocksStorageMovementResult#SUCCESS. Here,
-   * first occurrence is #blocksStorageMovementUnReportedItemsCheck() and then
-   * #blockStorageMovementResultCheck().
+   * Partial block movement. Here, first occurrence is
+   * #blocksStorageMovementUnReportedItemsCheck() and then
+   * #blockStorageMovementReportedItemsCheck().
    */
   @Test(timeout = 30000)
   public void testPartialBlockMovementShouldBeRetried2() throws Exception {
     Long item = new Long(1234);
-    bsmAttemptedItems.add(new ItemInfo(0L, item), false);
-    bsmAttemptedItems.addResults(
-        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
-            item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
+    Long trackID = 0L;
+    List<Block> blocks = new ArrayList<>();
+    blocks.add(new Block(item));
+    bsmAttemptedItems
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks));
+    Block[] blksMovementReport = new Block[1];
+    blksMovementReport[0] = new Block(item);
+    bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
 
     Thread.sleep(selfRetryTimeout * 2); // Waiting to get timed out
 
     bsmAttemptedItems.blocksStorageMovementUnReportedItemsCheck();
-    bsmAttemptedItems.blockStorageMovementResultCheck();
+    bsmAttemptedItems.blockStorageMovementReportedItemsCheck();
 
     assertTrue("Failed to add to the retry list",
-        checkItemMovedForRetry(item, 5000));
+        checkItemMovedForRetry(trackID, 5000));
     assertEquals("Failed to remove from the attempted list", 0,
         bsmAttemptedItems.getAttemptedItemsCount());
   }
 
   /**
-   * Partial block movement with only BlocksStorageMovementResult#FAILURE
-   * result and storageMovementAttemptedItems list is empty.
+   * Partial block movement with only BlocksStorageMoveAttemptFinished report
+   * and storageMovementAttemptedItems list is empty.
    */
   @Test(timeout = 30000)
   public void testPartialBlockMovementWithEmptyAttemptedQueue()
       throws Exception {
     Long item = new Long(1234);
-    bsmAttemptedItems.addResults(
-        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
-            item, BlocksStorageMovementResult.Status.FAILURE)});
-    bsmAttemptedItems.blockStorageMovementResultCheck();
+    Long trackID = 0L;
+    List<Block> blocks = new ArrayList<>();
+    blocks.add(new Block(item));
+    bsmAttemptedItems
+        .add(new AttemptedItemInfo(trackID, trackID, 0L, blocks));
+    Block[] blksMovementReport = new Block[1];
+    blksMovementReport[0] = new Block(item);
+    bsmAttemptedItems.addReportedMovedBlocks(blksMovementReport);
     assertFalse(
         "Should not add in queue again if it is not there in"
             + " storageMovementAttemptedItems",
-        checkItemMovedForRetry(item, 5000));
-    assertEquals("Failed to remove from the attempted list", 0,
-        bsmAttemptedItems.getAttemptedItemsCount());
-  }
-
-  /**
-   * Partial block movement with BlocksStorageMovementResult#FAILURE result and
-   * storageMovementAttemptedItems.
-   */
-  @Test(timeout = 30000)
-  public void testPartialBlockMovementShouldBeRetried4() throws Exception {
-    Long item = new Long(1234);
-    bsmAttemptedItems.add(new ItemInfo(0L, item), false);
-    bsmAttemptedItems.addResults(
-        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
-            item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
-    bsmAttemptedItems.blockStorageMovementResultCheck();
-    assertTrue("Failed to add to the retry list",
-        checkItemMovedForRetry(item, 5000));
-    assertEquals("Failed to remove from the attempted list", 0,
+        checkItemMovedForRetry(trackID, 5000));
+    assertEquals("Failed to remove from the attempted list", 1,
         bsmAttemptedItems.getAttemptedItemsCount());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 36beaa8..65628b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.InternalDataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -141,7 +141,7 @@ public class TestDeadDatanode {
     DatanodeCommand[] cmd =
         dnp.sendHeartbeat(reg, rep, 0L, 0L, 0, 0, 0, null, true,
             SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
-            new BlocksStorageMovementResult[0]).getCommands();
+            new BlocksStorageMoveAttemptFinished(null)).getCommands();
     assertEquals(1, cmd.length);
     assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());


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


[36/50] [abbrv] hadoop git commit: HDFS-11874. [SPS]: Document the SPS feature. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11874. [SPS]: Document the SPS feature. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: 4d174a2d7e8e45cdb4a1cfcfe3d68be0307d5b98
Parents: 85ae3a5
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Jul 14 22:36:09 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:29 2017 +0530

----------------------------------------------------------------------
 .../src/site/markdown/ArchivalStorage.md        | 51 ++++++++++++++++++--
 1 file changed, 48 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d174a2d/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 f1895fc..668bb20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -95,8 +95,44 @@ The effective storage policy can be retrieved by the "[`storagepolicies -getStor
 
     The default storage type of a datanode storage location will be DISK if it does not have a storage type tagged explicitly.
 
-Mover - A New Data Migration Tool
----------------------------------
+Storage Policy Based Data Movement
+----------------------------------
+
+Setting a new storage policy on already existing file/dir will change the policy in Namespace, but it will not move the blocks physically across storage medias.
+Following 2 options will allow users to move the blocks based on new policy set. So, once user change/set to a new policy on file/directory, user should also perform one of the following options to achieve the desired data movement. Note that both options cannot be allowed to run simultaneously.
+
+### <u>S</u>torage <u>P</u>olicy <u>S</u>atisfier (SPS)
+
+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.
+
+Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HDFS-10285)](https://issues.apache.org/jira/browse/HDFS-10285)
+
+* **Note**: When user invokes `satisfyStoragePolicy()` API on a directory, SPS will consider the files which are immediate to that directory. Sub-directories won't be considered for satisfying the policy. Its user responsibility to call this API on directories recursively, to track all files under the sub tree.
+
+* HdfsAdmin API :
+        `public void satisfyStoragePolicy(final Path path) throws IOException`
+
+* Arguments :
+
+| | |
+|:---- |:---- |
+| `path` | A path which requires blocks storage movement. |
+
+####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.recheck.timeout.millis** - A timeout to re-check the processed block storage movement
+   command results from Co-ordinator Datanode.
+
+*   **dfs.storage.policy.satisfier.self.retry.timeout.millis** - A timeout to retry if no block movement results reported from
+   Co-ordinator Datanode in this configured timeout.
+
+### Mover - A New Data Migration Tool
 
 A new data migration tool is added for archiving data. The tool is similar to Balancer. It periodically scans the files in HDFS to check if the block placement satisfies the storage policy. For the blocks violating the storage policy, it moves the replicas to a different storage type in order to fulfill the storage policy requirement. Note that it always tries to move block replicas within the same node whenever possible. If that is not possible (e.g. when a node doesn’t have the target storage type) then it will copy the block replicas to another node over the network.
 
@@ -113,6 +149,10 @@ A new data migration tool is added for archiving data. The tool is similar to Ba
 
 Note that, when both -p and -f options are omitted, the default path is the root directory.
 
+####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.
+
 Storage Policy Commands
 -----------------------
 
@@ -171,7 +211,8 @@ Get the storage policy of a file or a directory.
 
 ### Satisfy Storage Policy
 
-Schedule blocks to move based on file/directory policy. This command applicable only to the given path and its immediate children. Sub-directories won't be considered for satisfying the policy.
+Schedule blocks to move based on file's/directory's current storage policy.
+Note: For directory case, it will consider immediate files under that directory and it will not consider sub directories recursively.
 
 * Command:
 
@@ -191,4 +232,8 @@ Check the running status of Storage Policy Satisfier in namenode. If it is runni
 
         hdfs storagepolicies -isSPSRunning
 
+### 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
+
++       hdfs dfsadmin -reconfig namenode <host:ipc_port> start
 


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


[41/50] [abbrv] hadoop git commit: HDFS-12225: [SPS]: Optimize extended attributes for tracking SPS movements. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-12225: [SPS]: Optimize extended attributes for tracking SPS movements. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: b58239d36a6986c46141d290020b59eb6f3b2c9d
Parents: 9395fcf
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Aug 23 15:37:03 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:31:02 2017 +0530

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |  21 +-
 .../server/blockmanagement/DatanodeManager.java |  14 +-
 .../hdfs/server/datanode/BPOfferService.java    |   1 +
 .../BlockStorageMovementAttemptedItems.java     |  95 +++++---
 .../namenode/BlockStorageMovementNeeded.java    | 233 ++++++++++++++++++-
 .../namenode/FSDirSatisfyStoragePolicyOp.java   |  91 +++-----
 .../hdfs/server/namenode/FSDirXAttrOp.java      |  11 +-
 .../hdfs/server/namenode/FSDirectory.java       |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   2 +-
 .../server/namenode/StoragePolicySatisfier.java | 108 ++++++---
 .../TestStoragePolicySatisfyWorker.java         |   5 +-
 .../TestBlockStorageMovementAttemptedItems.java |  34 +--
 .../TestPersistentStoragePolicySatisfier.java   | 104 +++++++++
 .../namenode/TestStoragePolicySatisfier.java    | 127 +++++-----
 14 files changed, 589 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 68fbf6a..2748544 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
@@ -89,7 +89,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.server.namenode.BlockStorageMovementNeeded;
 import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@@ -436,9 +435,6 @@ public class BlockManager implements BlockStatsMXBean {
   private final StoragePolicySatisfier sps;
   private final boolean storagePolicyEnabled;
   private boolean spsEnabled;
-  private final BlockStorageMovementNeeded storageMovementNeeded =
-      new BlockStorageMovementNeeded();
-
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
    */
@@ -481,8 +477,7 @@ public class BlockManager implements BlockStatsMXBean {
         conf.getBoolean(
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT);
-    sps = new StoragePolicySatisfier(namesystem, storageMovementNeeded, this,
-        conf);
+    sps = new StoragePolicySatisfier(namesystem, this, conf);
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     this.maxCorruptFilesReturned = conf.getInt(
@@ -4939,20 +4934,6 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Set file block collection for which storage movement needed for its blocks.
-   *
-   * @param id
-   *          - file block collection id.
-   */
-  public void satisfyStoragePolicy(long id) {
-    storageMovementNeeded.add(id);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Added block collection id {} to block "
-          + "storageMovementNeeded queue", id);
-    }
-  }
-
-  /**
    * Gets the storage policy satisfier instance.
    *
    * @return sps

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index a298843..3504cb0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1749,6 +1749,13 @@ public class DatanodeManager {
       }
     }
 
+    if (nodeinfo.shouldDropSPSWork()) {
+      cmds.add(DropSPSWorkCommand.DNA_DROP_SPS_WORK_COMMAND);
+      // Set back to false to indicate that the new value has been sent to the
+      // datanode.
+      nodeinfo.setDropSPSWork(false);
+    }
+
     // check pending block storage movement tasks
     BlockStorageMovementInfosBatch blkStorageMovementInfosBatch = nodeinfo
         .getBlocksToMoveStorages();
@@ -1760,13 +1767,6 @@ public class DatanodeManager {
           blkStorageMovementInfosBatch.getBlockMovingInfo()));
     }
 
-    if (nodeinfo.shouldDropSPSWork()) {
-      cmds.add(DropSPSWorkCommand.DNA_DROP_SPS_WORK_COMMAND);
-      // Set back to false to indicate that the new value has been sent to the
-      // datanode.
-      nodeinfo.setDropSPSWork(false);
-    }
-
     if (!cmds.isEmpty()) {
       return cmds.toArray(new DatanodeCommand[cmds.size()]);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 39ff4b9..d60fb6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -823,6 +823,7 @@ class BPOfferService {
     case DatanodeProtocol.DNA_UNCACHE:
     case DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION:
     case DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT:
+    case DatanodeProtocol.DNA_DROP_SPS_WORK_COMMAND:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       break;
     default:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 37833e2..278b62b 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
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult.Status;
 import org.apache.hadoop.util.Daemon;
@@ -54,7 +55,7 @@ public class BlockStorageMovementAttemptedItems {
    * A map holds the items which are already taken for blocks movements
    * processing and sent to DNs.
    */
-  private final Map<Long, ItemInfo> storageMovementAttemptedItems;
+  private final Map<Long, AttemptedItemInfo> storageMovementAttemptedItems;
   private final List<BlocksStorageMovementResult> storageMovementAttemptedResults;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
@@ -91,18 +92,19 @@ public class BlockStorageMovementAttemptedItems {
    * Add item to block storage movement attempted items map which holds the
    * tracking/blockCollection id versus time stamp.
    *
-   * @param blockCollectionID
-   *          - tracking id / block collection id
+   * @param itemInfo
+   *          - tracking info
    * @param allBlockLocsAttemptedToSatisfy
-   *          - failed to find matching target nodes to satisfy storage type for
-   *          all the block locations of the given blockCollectionID
+   *          - failed to find matching target nodes to satisfy storage type
+   *          for all the block locations of the given blockCollectionID
    */
-  public void add(Long blockCollectionID,
-      boolean allBlockLocsAttemptedToSatisfy) {
+  public void add(ItemInfo itemInfo, boolean allBlockLocsAttemptedToSatisfy) {
     synchronized (storageMovementAttemptedItems) {
-      ItemInfo itemInfo = new ItemInfo(monotonicNow(),
+      AttemptedItemInfo attemptedItemInfo = new AttemptedItemInfo(
+          itemInfo.getRootId(), itemInfo.getTrackId(), monotonicNow(),
           allBlockLocsAttemptedToSatisfy);
-      storageMovementAttemptedItems.put(blockCollectionID, itemInfo);
+      storageMovementAttemptedItems.put(itemInfo.getTrackId(),
+          attemptedItemInfo);
     }
   }
 
@@ -167,21 +169,27 @@ public class BlockStorageMovementAttemptedItems {
    * satisfy storage policy. This is used by
    * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
    */
-  private final static class ItemInfo {
+  private final static class AttemptedItemInfo extends ItemInfo {
     private long lastAttemptedOrReportedTime;
     private final boolean allBlockLocsAttemptedToSatisfy;
 
     /**
-     * ItemInfo constructor.
+     * AttemptedItemInfo constructor.
      *
+     * @param rootId
+     *          rootId for trackId
+     * @param trackId
+     *          trackId for file.
      * @param lastAttemptedOrReportedTime
      *          last attempted or reported time
      * @param allBlockLocsAttemptedToSatisfy
      *          whether all the blocks in the trackID were attempted and blocks
      *          movement has been scheduled to satisfy storage policy
      */
-    private ItemInfo(long lastAttemptedOrReportedTime,
+    private AttemptedItemInfo(long rootId, long trackId,
+        long lastAttemptedOrReportedTime,
         boolean allBlockLocsAttemptedToSatisfy) {
+      super(rootId, trackId);
       this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
       this.allBlockLocsAttemptedToSatisfy = allBlockLocsAttemptedToSatisfy;
     }
@@ -211,6 +219,7 @@ public class BlockStorageMovementAttemptedItems {
     private void touchLastReportedTimeStamp() {
       this.lastAttemptedOrReportedTime = monotonicNow();
     }
+
   }
 
   /**
@@ -239,18 +248,20 @@ public class BlockStorageMovementAttemptedItems {
   @VisibleForTesting
   void blocksStorageMovementUnReportedItemsCheck() {
     synchronized (storageMovementAttemptedItems) {
-      Iterator<Entry<Long, ItemInfo>> iter = storageMovementAttemptedItems
-          .entrySet().iterator();
+      Iterator<Entry<Long, AttemptedItemInfo>> iter =
+          storageMovementAttemptedItems.entrySet().iterator();
       long now = monotonicNow();
       while (iter.hasNext()) {
-        Entry<Long, ItemInfo> entry = iter.next();
-        ItemInfo itemInfo = entry.getValue();
+        Entry<Long, AttemptedItemInfo> entry = iter.next();
+        AttemptedItemInfo itemInfo = entry.getValue();
         if (now > itemInfo.getLastAttemptedOrReportedTime()
             + selfRetryTimeout) {
           Long blockCollectionID = entry.getKey();
           synchronized (storageMovementAttemptedResults) {
             if (!isExistInResult(blockCollectionID)) {
-              blockStorageMovementNeeded.add(blockCollectionID);
+              ItemInfo candidate = new ItemInfo(
+                  itemInfo.getRootId(), blockCollectionID);
+              blockStorageMovementNeeded.add(candidate);
               iter.remove();
               LOG.info("TrackID: {} becomes timed out and moved to needed "
                   + "retries queue for next iteration.", blockCollectionID);
@@ -297,17 +308,30 @@ public class BlockStorageMovementAttemptedItems {
         synchronized (storageMovementAttemptedItems) {
           Status status = storageMovementAttemptedResult.getStatus();
           long trackId = storageMovementAttemptedResult.getTrackId();
-          ItemInfo itemInfo;
+          AttemptedItemInfo attemptedItemInfo = storageMovementAttemptedItems
+              .get(trackId);
+          // itemInfo is null means no root for trackId, using trackId only as
+          // root and handling it in
+          // blockStorageMovementNeeded#removeIteamTrackInfo() for cleaning
+          // the xAttr
+          ItemInfo itemInfo = new ItemInfo((attemptedItemInfo != null)
+              ? attemptedItemInfo.getRootId() : trackId, trackId);
           switch (status) {
           case FAILURE:
-            blockStorageMovementNeeded.add(trackId);
-            LOG.warn("Blocks storage movement results for the tracking id: {}"
-                + " is reported from co-ordinating datanode, but result"
-                + " status is FAILURE. So, added for retry", trackId);
+            if (attemptedItemInfo != null) {
+              blockStorageMovementNeeded.add(itemInfo);
+              LOG.warn("Blocks storage movement results for the tracking id:"
+                  + "{} is reported from co-ordinating datanode, but result"
+                  + " status is FAILURE. So, added for retry", trackId);
+            } else {
+              LOG.info("Blocks storage movement is FAILURE for the track"
+                  + " id {}. But the trackID doesn't exists in"
+                  + " storageMovementAttemptedItems list.", trackId);
+              blockStorageMovementNeeded
+                  .removeItemTrackInfo(itemInfo);
+            }
             break;
           case SUCCESS:
-            itemInfo = storageMovementAttemptedItems.get(trackId);
-
             // ItemInfo could be null. One case is, before the blocks movements
             // result arrives the attempted trackID became timed out and then
             // removed the trackID from the storageMovementAttemptedItems list.
@@ -318,33 +342,32 @@ public class BlockStorageMovementAttemptedItems {
             // storage policy in previous SPS iteration.
             String msg = "Blocks storage movement is SUCCESS for the track id: "
                 + trackId + " reported from co-ordinating datanode.";
-            if (itemInfo != null) {
-              if (!itemInfo.isAllBlockLocsAttemptedToSatisfy()) {
-                blockStorageMovementNeeded.add(trackId);
+            if (attemptedItemInfo != null) {
+              if (!attemptedItemInfo.isAllBlockLocsAttemptedToSatisfy()) {
+                blockStorageMovementNeeded
+                    .add(new ItemInfo(attemptedItemInfo.getRootId(), trackId));
                 LOG.warn("{} But adding trackID back to retry queue as some of"
                     + " the blocks couldn't find matching target nodes in"
                     + " previous SPS iteration.", msg);
               } else {
                 LOG.info(msg);
-                // Remove xattr for the track id.
-                this.sps.postBlkStorageMovementCleanup(
-                    storageMovementAttemptedResult.getTrackId());
+                blockStorageMovementNeeded
+                    .removeItemTrackInfo(itemInfo);
               }
             } else {
               LOG.info("{} But the trackID doesn't exists in "
                   + "storageMovementAttemptedItems list", msg);
-              // Remove xattr for the track id.
-              this.sps.postBlkStorageMovementCleanup(
-                  storageMovementAttemptedResult.getTrackId());
+              blockStorageMovementNeeded
+              .removeItemTrackInfo(itemInfo);
             }
             break;
           case IN_PROGRESS:
             isInprogress = true;
-            itemInfo = storageMovementAttemptedItems
+            attemptedItemInfo = storageMovementAttemptedItems
                 .get(storageMovementAttemptedResult.getTrackId());
-            if(itemInfo != null){
+            if(attemptedItemInfo != null){
               // update the attempted expiration time to next cycle.
-              itemInfo.touchLastReportedTimeStamp();
+              attemptedItemInfo.touchLastReportedTimeStamp();
             }
             break;
           default:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
index 3241e6d..41a3a6c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
@@ -17,28 +17,86 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * A Class to track the block collection IDs for which physical storage movement
- * needed as per the Namespace and StorageReports from DN.
+ * A Class to track the block collection IDs (Inode's ID) for which physical
+ * storage movement needed as per the Namespace and StorageReports from DN.
+ * It scan the pending directories for which storage movement is required and
+ * schedule the block collection IDs for movement. It track the info of
+ * scheduled items and remove the SPS xAttr from the file/Directory once
+ * movement is success.
  */
 @InterfaceAudience.Private
 public class BlockStorageMovementNeeded {
-  private final Queue<Long> storageMovementNeeded = new LinkedList<Long>();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockStorageMovementNeeded.class);
+
+  private final Queue<ItemInfo> storageMovementNeeded =
+      new LinkedList<ItemInfo>();
 
   /**
-   * Add the block collection id to tracking list for which storage movement
+   * Map of rootId and number of child's. Number of child's indicate the number
+   * of files pending to satisfy the policy.
+   */
+  private final Map<Long, Integer> pendingWorkForDirectory =
+      new HashMap<Long, Integer>();
+
+  private final Namesystem namesystem;
+
+  // List of pending dir to satisfy the policy
+  private final Queue<Long> spsDirsToBeTraveresed = new LinkedList<Long>();
+
+  private final StoragePolicySatisfier sps;
+
+  private Daemon fileInodeIdCollector;
+
+  public BlockStorageMovementNeeded(Namesystem namesystem,
+      StoragePolicySatisfier sps) {
+    this.namesystem = namesystem;
+    this.sps = sps;
+  }
+
+  /**
+   * Add the candidate to tracking list for which storage movement
    * expected if necessary.
    *
-   * @param blockCollectionID
-   *          - block collection id, which is nothing but inode id.
+   * @param trackInfo
+   *          - track info for satisfy the policy
    */
-  public synchronized void add(Long blockCollectionID) {
-    storageMovementNeeded.add(blockCollectionID);
+  public synchronized void add(ItemInfo trackInfo) {
+    storageMovementNeeded.add(trackInfo);
+  }
+
+  /**
+   * Add the itemInfo to tracking list for which storage movement
+   * expected if necessary.
+   * @param rootId
+   *            - root inode id
+   * @param itemInfoList
+   *            - List of child in the directory
+   */
+  private synchronized void addAll(Long rootId,
+      List<ItemInfo> itemInfoList) {
+    storageMovementNeeded.addAll(itemInfoList);
+    pendingWorkForDirectory.put(rootId, itemInfoList.size());
   }
 
   /**
@@ -47,11 +105,168 @@ public class BlockStorageMovementNeeded {
    *
    * @return block collection ID
    */
-  public synchronized Long get() {
+  public synchronized ItemInfo get() {
     return storageMovementNeeded.poll();
   }
 
+  public synchronized void addToPendingDirQueue(long id) {
+    spsDirsToBeTraveresed.add(id);
+    // Notify waiting FileInodeIdCollector thread about the newly
+    // added SPS path.
+    synchronized (spsDirsToBeTraveresed) {
+      spsDirsToBeTraveresed.notify();
+    }
+  }
+
   public synchronized void clearAll() {
+    spsDirsToBeTraveresed.clear();
     storageMovementNeeded.clear();
+    pendingWorkForDirectory.clear();
+  }
+
+  /**
+   * Decrease the pending child count for directory once one file blocks moved
+   * successfully. Remove the SPS xAttr if pending child count is zero.
+   */
+  public synchronized void removeItemTrackInfo(ItemInfo trackInfo)
+      throws IOException {
+    if (trackInfo.isDir()) {
+      // If track is part of some root then reduce the pending directory work
+      // count.
+      long rootId = trackInfo.getRootId();
+      INode inode = namesystem.getFSDirectory().getInode(rootId);
+      if (inode == null) {
+        // directory deleted just remove it.
+        this.pendingWorkForDirectory.remove(rootId);
+      } else {
+        if (pendingWorkForDirectory.get(rootId) != null) {
+          Integer pendingWork = pendingWorkForDirectory.get(rootId) - 1;
+          pendingWorkForDirectory.put(rootId, pendingWork);
+          if (pendingWork <= 0) {
+            namesystem.removeXattr(rootId, XATTR_SATISFY_STORAGE_POLICY);
+            pendingWorkForDirectory.remove(rootId);
+          }
+        }
+      }
+    } else {
+      // Remove xAttr if trackID doesn't exist in
+      // storageMovementAttemptedItems or file policy satisfied.
+      namesystem.removeXattr(trackInfo.getTrackId(),
+          XATTR_SATISFY_STORAGE_POLICY);
+    }
+  }
+
+  public synchronized void clearQueue(long trackId) {
+    spsDirsToBeTraveresed.remove(trackId);
+    Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
+    while (iterator.hasNext()) {
+      ItemInfo next = iterator.next();
+      if (next.getRootId() == trackId) {
+        iterator.remove();
+      }
+    }
+    pendingWorkForDirectory.remove(trackId);
+  }
+
+  /**
+   * Clean all the movements in spsDirsToBeTraveresed/storageMovementNeeded
+   * and notify to clean up required resources.
+   * @throws IOException
+   */
+  public synchronized void clearQueuesWithNotification() {
+    // Remove xAttr from directories
+    Long trackId;
+    while ((trackId = spsDirsToBeTraveresed.poll()) != null) {
+      try {
+        // Remove xAttr for file
+        namesystem.removeXattr(trackId, XATTR_SATISFY_STORAGE_POLICY);
+      } catch (IOException ie) {
+        LOG.warn("Failed to remove SPS xattr for track id " + trackId, ie);
+      }
+    }
+
+    // File's directly added to storageMovementNeeded, So try to remove
+    // xAttr for file
+    ItemInfo itemInfo;
+    while ((itemInfo = storageMovementNeeded.poll()) != null) {
+      try {
+        // Remove xAttr for file
+        if (!itemInfo.isDir()) {
+          namesystem.removeXattr(itemInfo.getTrackId(),
+              XATTR_SATISFY_STORAGE_POLICY);
+        }
+      } catch (IOException ie) {
+        LOG.warn(
+            "Failed to remove SPS xattr for track id "
+                + itemInfo.getTrackId(), ie);
+      }
+    }
+    this.clearAll();
+  }
+
+  /**
+   * Take dir tack ID from the spsDirsToBeTraveresed queue and collect child
+   * ID's to process for satisfy the policy.
+   */
+  private class FileInodeIdCollector implements Runnable {
+    @Override
+    public void run() {
+      LOG.info("Starting FileInodeIdCollector!.");
+      while (namesystem.isRunning() && sps.isRunning()) {
+        try {
+          if (!namesystem.isInSafeMode()) {
+            FSDirectory fsd = namesystem.getFSDirectory();
+            Long rootINodeId = spsDirsToBeTraveresed.poll();
+            if (rootINodeId == null) {
+              // Waiting for SPS path
+              synchronized (spsDirsToBeTraveresed) {
+                spsDirsToBeTraveresed.wait(5000);
+              }
+            } else {
+              INode rootInode = fsd.getInode(rootINodeId);
+              if (rootInode != null) {
+                // TODO : HDFS-12291
+                // 1. Implement an efficient recursive directory iteration
+                // mechanism and satisfies storage policy for all the files
+                // under the given directory.
+                // 2. Process files in batches,so datanodes workload can be
+                // handled.
+                List<ItemInfo> itemInfoList =
+                    new ArrayList<>();
+                for (INode childInode : rootInode.asDirectory()
+                    .getChildrenList(Snapshot.CURRENT_STATE_ID)) {
+                  if (childInode.isFile()
+                      && childInode.asFile().numBlocks() != 0) {
+                    itemInfoList.add(
+                        new ItemInfo(rootINodeId, childInode.getId()));
+                  }
+                }
+                if (itemInfoList.isEmpty()) {
+                  // satisfy track info is empty, so remove the xAttr from the
+                  // directory
+                  namesystem.removeXattr(rootINodeId,
+                      XATTR_SATISFY_STORAGE_POLICY);
+                }
+                addAll(rootINodeId, itemInfoList);
+              }
+            }
+          }
+        } catch (Throwable t) {
+          LOG.warn("Exception while loading inodes to satisfy the policy", t);
+        }
+      }
+    }
+  }
+
+  public void start() {
+    fileInodeIdCollector = new Daemon(new FileInodeIdCollector());
+    fileInodeIdCollector.setName("FileInodeIdCollector");
+    fileInodeIdCollector.start();
+  }
+
+  public void stop() {
+    if (fileInodeIdCollector != null) {
+      fileInodeIdCollector.interrupt();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
index bd4e5ed..fb6eec9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
 
@@ -31,6 +30,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.collect.Lists;
 
@@ -60,10 +60,24 @@ final class FSDirSatisfyStoragePolicyOp {
       if (fsd.isPermissionEnabled()) {
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
       }
-      XAttr satisfyXAttr = unprotectedSatisfyStoragePolicy(iip, bm, fsd);
-      if (satisfyXAttr != null) {
+      INode inode = FSDirectory.resolveLastINode(iip);
+      if (inodeHasSatisfyXAttr(inode)) {
+        throw new IOException(
+            "Cannot request to call satisfy storage policy on path "
+                + inode.getFullPathName()
+                + ", as this file/dir was already called for satisfying "
+                + "storage policy.");
+      }
+      if (unprotectedSatisfyStoragePolicy(inode, fsd)) {
+        XAttr satisfyXAttr = XAttrHelper
+            .buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
         List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
         xAttrs.add(satisfyXAttr);
+        List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
+        List<XAttr> newXAttrs = FSDirXAttrOp.setINodeXAttrs(fsd, existingXAttrs,
+            xAttrs, EnumSet.of(XAttrSetFlag.CREATE));
+        XAttrStorage.updateINodeXAttrs(inode, newXAttrs,
+            iip.getLatestSnapshotId());
         fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
       }
     } finally {
@@ -72,62 +86,29 @@ final class FSDirSatisfyStoragePolicyOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  static XAttr unprotectedSatisfyStoragePolicy(INodesInPath iip,
-      BlockManager bm, FSDirectory fsd) throws IOException {
-
-    final INode inode = FSDirectory.resolveLastINode(iip);
-    final int snapshotId = iip.getLatestSnapshotId();
-    final List<INode> candidateNodes = new ArrayList<>();
-
-    // TODO: think about optimization here, label the dir instead
-    // of the sub-files of the dir.
+  static boolean unprotectedSatisfyStoragePolicy(INode inode, FSDirectory fsd) {
     if (inode.isFile() && inode.asFile().numBlocks() != 0) {
-      candidateNodes.add(inode);
-    } else if (inode.isDirectory()) {
-      for (INode node : inode.asDirectory().getChildrenList(snapshotId)) {
-        if (node.isFile() && node.asFile().numBlocks() != 0) {
-          candidateNodes.add(node);
-        }
-      }
-    }
-
-    if (candidateNodes.isEmpty()) {
-      return null;
+      // Adding directly in the storageMovementNeeded queue, So it can
+      // get more priority compare to directory.
+      fsd.getBlockManager().getStoragePolicySatisfier()
+          .satisfyStoragePolicy(inode.getId());
+      return true;
+    } else if (inode.isDirectory()
+        && inode.asDirectory().getChildrenNum(Snapshot.CURRENT_STATE_ID) > 0) {
+      // Adding directory in the pending queue, so FileInodeIdCollector process
+      // directory child in batch and recursively
+      fsd.getBlockManager().getStoragePolicySatisfier()
+          .addInodeToPendingDirQueue(inode.getId());
+      return true;
     }
-    // If node has satisfy xattr, then stop adding it
-    // to satisfy movement queue.
-    if (inodeHasSatisfyXAttr(candidateNodes)) {
-      throw new IOException(
-          "Cannot request to call satisfy storage policy on path "
-              + iip.getPath()
-              + ", as this file/dir was already called for satisfying "
-              + "storage policy.");
-    }
-
-    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
-    final XAttr satisfyXAttr = XAttrHelper
-        .buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
-    xattrs.add(satisfyXAttr);
-
-    for (INode node : candidateNodes) {
-      bm.satisfyStoragePolicy(node.getId());
-      List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(node);
-      List<XAttr> newXAttrs = FSDirXAttrOp.setINodeXAttrs(fsd, existingXAttrs,
-          xattrs, EnumSet.of(XAttrSetFlag.CREATE));
-      XAttrStorage.updateINodeXAttrs(node, newXAttrs, snapshotId);
-    }
-    return satisfyXAttr;
+    return false;
   }
 
-  private static boolean inodeHasSatisfyXAttr(List<INode> candidateNodes) {
-    // If the node is a directory and one of the child files
-    // has satisfy xattr, then return true for this directory.
-    for (INode inode : candidateNodes) {
-      final XAttrFeature f = inode.getXAttrFeature();
-      if (inode.isFile() && f != null
-          && f.getXAttr(XATTR_SATISFY_STORAGE_POLICY) != null) {
-        return true;
-      }
+  private static boolean inodeHasSatisfyXAttr(INode inode) {
+    final XAttrFeature f = inode.getXAttrFeature();
+    if (inode.isFile() && f != null
+        && f.getXAttr(XATTR_SATISFY_STORAGE_POLICY) != null) {
+      return true;
     }
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
index 488cc9d..62d36c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
@@ -200,6 +200,14 @@ class FSDirXAttrOp {
     List<XAttr> newXAttrs = filterINodeXAttrs(existingXAttrs, toRemove,
                                               removedXAttrs);
     if (existingXAttrs.size() != newXAttrs.size()) {
+      for (XAttr xattr : toRemove) {
+        if (XATTR_SATISFY_STORAGE_POLICY
+            .equals(XAttrHelper.getPrefixedName(xattr))) {
+          fsd.getBlockManager().getStoragePolicySatisfier()
+              .clearQueue(inode.getId());
+          break;
+        }
+      }
       XAttrStorage.updateINodeXAttrs(inode, newXAttrs, snapshotId);
       return removedXAttrs;
     }
@@ -291,8 +299,7 @@ class FSDirXAttrOp {
 
       // Add inode id to movement queue if xattrs contain satisfy xattr.
       if (XATTR_SATISFY_STORAGE_POLICY.equals(xaName)) {
-        FSDirSatisfyStoragePolicyOp.unprotectedSatisfyStoragePolicy(iip,
-            fsd.getBlockManager(), fsd);
+        FSDirSatisfyStoragePolicyOp.unprotectedSatisfyStoragePolicy(inode, fsd);
         continue;
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 37ed027..848f6a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -1413,7 +1413,7 @@ public class FSDirectory implements Closeable {
     if (xattr == null) {
       return;
     }
-    getBlockManager().satisfyStoragePolicy(inode.getId());
+    FSDirSatisfyStoragePolicyOp.unprotectedSatisfyStoragePolicy(inode, this);
   }
 
   private void addEncryptionZone(INodeWithAdditionalFields inode,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 33d7b92..1e09a4d 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
@@ -1316,7 +1316,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       if (blockManager != null) {
-        blockManager.stopSPS(true);
+        blockManager.stopSPS(false);
       }
       stopSecretManager();
       leaseManager.stopMonitor();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 3165813..48d0598 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
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
-
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -106,10 +103,10 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   public StoragePolicySatisfier(final Namesystem namesystem,
-      final BlockStorageMovementNeeded storageMovementNeeded,
       final BlockManager blkManager, Configuration conf) {
     this.namesystem = namesystem;
-    this.storageMovementNeeded = storageMovementNeeded;
+    this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem,
+        this);
     this.blockManager = blkManager;
     this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(
         conf.getLong(
@@ -146,7 +143,7 @@ public class StoragePolicySatisfier implements Runnable {
     // Ensure that all the previously submitted block movements(if any) have to
     // be stopped in all datanodes.
     addDropSPSWorkCommandsToAllDNs();
-
+    storageMovementNeeded.start();
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
@@ -162,14 +159,17 @@ public class StoragePolicySatisfier implements Runnable {
    */
   public synchronized void disable(boolean forceStop) {
     isRunning = false;
+
     if (storagePolicySatisfierThread == null) {
       return;
     }
 
+    storageMovementNeeded.stop();
+
     storagePolicySatisfierThread.interrupt();
     this.storageMovementsMonitor.stop();
     if (forceStop) {
-      this.clearQueuesWithNotification();
+      storageMovementNeeded.clearQueuesWithNotification();
       addDropSPSWorkCommandsToAllDNs();
     } else {
       LOG.info("Stopping StoragePolicySatisfier.");
@@ -184,6 +184,7 @@ public class StoragePolicySatisfier implements Runnable {
       disable(true);
     }
     this.storageMovementsMonitor.stopGracefully();
+
     if (storagePolicySatisfierThread == null) {
       return;
     }
@@ -220,10 +221,11 @@ public class StoragePolicySatisfier implements Runnable {
     while (namesystem.isRunning() && isRunning) {
       try {
         if (!namesystem.isInSafeMode()) {
-          Long blockCollectionID = storageMovementNeeded.get();
-          if (blockCollectionID != null) {
+          ItemInfo itemInfo = storageMovementNeeded.get();
+          if (itemInfo != null) {
+            long trackId = itemInfo.getTrackId();
             BlockCollection blockCollection =
-                namesystem.getBlockCollection(blockCollectionID);
+                namesystem.getBlockCollection(trackId);
             // Check blockCollectionId existence.
             if (blockCollection != null) {
               BlocksMovingAnalysisStatus status =
@@ -234,21 +236,21 @@ public class StoragePolicySatisfier implements Runnable {
                 // Just add to monitor, so it will be tracked for result and
                 // be removed on successful storage movement result.
               case ALL_BLOCKS_TARGETS_PAIRED:
-                this.storageMovementsMonitor.add(blockCollectionID, true);
+                this.storageMovementsMonitor.add(itemInfo, true);
                 break;
               // Add to monitor with allBlcoksAttemptedToSatisfy flag false, so
               // that it will be tracked and still it will be consider for retry
               // as analysis was not found targets for storage movement blocks.
               case FEW_BLOCKS_TARGETS_PAIRED:
-                this.storageMovementsMonitor.add(blockCollectionID, false);
+                this.storageMovementsMonitor.add(itemInfo, false);
                 break;
               case FEW_LOW_REDUNDANCY_BLOCKS:
                 if (LOG.isDebugEnabled()) {
-                  LOG.debug("Adding trackID " + blockCollectionID
+                  LOG.debug("Adding trackID " + trackId
                       + " back to retry queue as some of the blocks"
                       + " are low redundant.");
                 }
-                this.storageMovementNeeded.add(blockCollectionID);
+                this.storageMovementNeeded.add(itemInfo);
                 break;
               // Just clean Xattrs
               case BLOCKS_TARGET_PAIRING_SKIPPED:
@@ -256,9 +258,13 @@ public class StoragePolicySatisfier implements Runnable {
               default:
                 LOG.info("Block analysis skipped or blocks already satisfied"
                     + " with storages. So, Cleaning up the Xattrs.");
-                postBlkStorageMovementCleanup(blockCollectionID);
+                storageMovementNeeded.removeItemTrackInfo(itemInfo);
                 break;
               }
+            } else {
+              // File doesn't exists (maybe got deleted), remove trackId from
+              // the queue
+              storageMovementNeeded.removeItemTrackInfo(itemInfo);
             }
           }
         }
@@ -828,31 +834,63 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   /**
-   * Clean all the movements in storageMovementNeeded and notify
-   * to clean up required resources.
-   * @throws IOException
+   * Set file inode in queue for which storage movement needed for its blocks.
+   *
+   * @param inodeId
+   *          - file inode/blockcollection id.
    */
-  private void clearQueuesWithNotification() {
-    Long id;
-    while ((id = storageMovementNeeded.get()) != null) {
-      try {
-        postBlkStorageMovementCleanup(id);
-      } catch (IOException ie) {
-        LOG.warn("Failed to remove SPS "
-            + "xattr for collection id " + id, ie);
-      }
+  public void satisfyStoragePolicy(Long inodeId) {
+    //For file rootId and trackId is same
+    storageMovementNeeded.add(new ItemInfo(inodeId, inodeId));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added track info for inode {} to block "
+          + "storageMovementNeeded queue", inodeId);
     }
   }
 
+  public void addInodeToPendingDirQueue(long id) {
+    storageMovementNeeded.addToPendingDirQueue(id);
+  }
+
+  /**
+   * Clear queues for given track id.
+   */
+  public void clearQueue(long trackId) {
+    storageMovementNeeded.clearQueue(trackId);
+  }
+
   /**
-   * When block movement has been finished successfully, some additional
-   * operations should be notified, for example, SPS xattr should be
-   * removed.
-   * @param trackId track id i.e., block collection id.
-   * @throws IOException
+   * ItemInfo is a file info object for which need to satisfy the
+   * policy.
    */
-  public void postBlkStorageMovementCleanup(long trackId)
-      throws IOException {
-    this.namesystem.removeXattr(trackId, XATTR_SATISFY_STORAGE_POLICY);
+  public static class ItemInfo {
+    private long rootId;
+    private long trackId;
+
+    public ItemInfo(long rootId, long trackId) {
+      this.rootId = rootId;
+      this.trackId = trackId;
+    }
+
+    /**
+     * Return the root of the current track Id.
+     */
+    public long getRootId() {
+      return rootId;
+    }
+
+    /**
+     * Return the File inode Id for which needs to satisfy the policy.
+     */
+    public long getTrackId() {
+      return trackId;
+    }
+
+    /**
+     * Returns true if the tracking path is a directory, false otherwise.
+     */
+    public boolean isDir() {
+      return (rootId != trackId);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 402d4d1..b84b1d2 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
@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
@@ -115,9 +114,7 @@ public class TestStoragePolicySatisfyWorker {
     // move to ARCHIVE
     dfs.setStoragePolicy(new Path(file), "COLD");
 
-    FSNamesystem namesystem = cluster.getNamesystem();
-    INode inode = namesystem.getFSDirectory().getINode(file);
-    namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+    dfs.satisfyStoragePolicy(new Path(file));
 
     cluster.triggerHeartbeats();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 04a63ac..55ebf9c 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.apache.hadoop.util.Time.monotonicNow;
 import static org.junit.Assert.*;
 
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.junit.After;
 import org.junit.Before;
@@ -38,7 +39,9 @@ public class TestBlockStorageMovementAttemptedItems {
 
   @Before
   public void setup() throws Exception {
-    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded();
+    unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(
+        Mockito.mock(Namesystem.class),
+        Mockito.mock(StoragePolicySatisfier.class));
     StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
     bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
         selfRetryTimeout, unsatisfiedStorageMovementFiles, sps);
@@ -57,9 +60,9 @@ public class TestBlockStorageMovementAttemptedItems {
     long stopTime = monotonicNow() + (retryTimeout * 2);
     boolean isItemFound = false;
     while (monotonicNow() < (stopTime)) {
-      Long ele = null;
+      ItemInfo ele = null;
       while ((ele = unsatisfiedStorageMovementFiles.get()) != null) {
-        if (item.longValue() == ele.longValue()) {
+        if (item == ele.getTrackId()) {
           isItemFound = true;
           break;
         }
@@ -77,7 +80,7 @@ public class TestBlockStorageMovementAttemptedItems {
   public void testAddResultWithFailureResult() throws Exception {
     bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item, true);
+    bsmAttemptedItems.add(new ItemInfo(0L, item), true);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
             item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
@@ -88,7 +91,7 @@ public class TestBlockStorageMovementAttemptedItems {
   public void testAddResultWithSucessResult() throws Exception {
     bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item, true);
+    bsmAttemptedItems.add(new ItemInfo(0L, item), true);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
             item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
@@ -99,7 +102,7 @@ public class TestBlockStorageMovementAttemptedItems {
   public void testNoResultAdded() throws Exception {
     bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item, true);
+    bsmAttemptedItems.add(new ItemInfo(0L, item), true);
     // After self retry timeout, it should be added back for retry
     assertTrue("Failed to add to the retry list",
         checkItemMovedForRetry(item, 600));
@@ -115,7 +118,7 @@ public class TestBlockStorageMovementAttemptedItems {
   @Test(timeout = 30000)
   public void testPartialBlockMovementShouldBeRetried1() throws Exception {
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item, false);
+    bsmAttemptedItems.add(new ItemInfo(0L, item), false);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
             item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
@@ -136,7 +139,7 @@ public class TestBlockStorageMovementAttemptedItems {
   @Test(timeout = 30000)
   public void testPartialBlockMovementShouldBeRetried2() throws Exception {
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item, false);
+    bsmAttemptedItems.add(new ItemInfo(0L, item), false);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
             item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
@@ -153,17 +156,20 @@ public class TestBlockStorageMovementAttemptedItems {
   }
 
   /**
-   * Partial block movement with only BlocksStorageMovementResult#FAILURE result
-   * and storageMovementAttemptedItems list is empty.
+   * Partial block movement with only BlocksStorageMovementResult#FAILURE
+   * result and storageMovementAttemptedItems list is empty.
    */
   @Test(timeout = 30000)
-  public void testPartialBlockMovementShouldBeRetried3() throws Exception {
+  public void testPartialBlockMovementWithEmptyAttemptedQueue()
+      throws Exception {
     Long item = new Long(1234);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
-            item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
+            item, BlocksStorageMovementResult.Status.FAILURE)});
     bsmAttemptedItems.blockStorageMovementResultCheck();
-    assertTrue("Failed to add to the retry list",
+    assertFalse(
+        "Should not add in queue again if it is not there in"
+            + " storageMovementAttemptedItems",
         checkItemMovedForRetry(item, 5000));
     assertEquals("Failed to remove from the attempted list", 0,
         bsmAttemptedItems.getAttemptedItemsCount());
@@ -176,7 +182,7 @@ public class TestBlockStorageMovementAttemptedItems {
   @Test(timeout = 30000)
   public void testPartialBlockMovementShouldBeRetried4() throws Exception {
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item, false);
+    bsmAttemptedItems.add(new ItemInfo(0L, item), false);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
             item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 8516ea0..e7b9148 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
@@ -20,16 +20,22 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
+import com.google.common.base.Supplier;
+
 import java.io.IOException;
+import java.util.List;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 import static org.junit.Assert.*;
@@ -482,6 +488,104 @@ public class TestPersistentStoragePolicySatisfier {
   }
 
   /**
+   * Test SPS xAttr on directory. xAttr should be removed from the directory
+   * once all the files blocks moved to specific storage.
+   */
+  @Test(timeout = 300000)
+  public void testSPSxAttrWhenSpsCalledForDir() throws Exception {
+    try {
+      clusterSetUp();
+      Path parent = new Path("/parent");
+      // create parent dir
+      fs.mkdirs(parent);
+
+      // create 10 child files
+      for (int i = 0; i < 5; i++) {
+        DFSTestUtil.createFile(fs, new Path(parent, "f" + i), 1024, (short) 3,
+            0);
+      }
+
+      // Set storage policy for parent directory
+      fs.setStoragePolicy(parent, "COLD");
+
+      // Stop one DN so we can check the SPS xAttr for directory.
+      DataNodeProperties stopDataNode = cluster.stopDataNode(0);
+
+      fs.satisfyStoragePolicy(parent);
+
+      // Check xAttr for parent directory
+      FSNamesystem namesystem = cluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode("/parent");
+      XAttrFeature f = inode.getXAttrFeature();
+      assertTrue("SPS xAttr should be exist",
+          f.getXAttr(XATTR_SATISFY_STORAGE_POLICY) != null);
+
+      // check for the child, SPS xAttr should not be there
+      for (int i = 0; i < 5; i++) {
+        inode = namesystem.getFSDirectory().getINode("/parent/f" + i);
+        f = inode.getXAttrFeature();
+        assertTrue(f == null);
+      }
+
+      cluster.restartDataNode(stopDataNode, false);
+
+      // wait and check all the file block moved in ARCHIVE
+      for (int i = 0; i < 5; i++) {
+        DFSTestUtil.waitExpectedStorageType("/parent/f" + i,
+            StorageType.ARCHIVE, 3, 30000, cluster.getFileSystem());
+      }
+      DFSTestUtil.waitForXattrRemoved("/parent", XATTR_SATISFY_STORAGE_POLICY,
+          namesystem, 10000);
+    } finally {
+      clusterShutdown();
+    }
+
+  }
+
+  /**
+   * Test SPS xAttr on file. xAttr should be removed from the file
+   * once all the blocks moved to specific storage.
+   */
+  @Test(timeout = 300000)
+  public void testSPSxAttrWhenSpsCalledForFile() throws Exception {
+    try {
+      clusterSetUp();
+      Path file = new Path("/file");
+      DFSTestUtil.createFile(fs, file, 1024, (short) 3, 0);
+
+      // Set storage policy for file
+      fs.setStoragePolicy(file, "COLD");
+
+      // Stop one DN so we can check the SPS xAttr for file.
+      DataNodeProperties stopDataNode = cluster.stopDataNode(0);
+
+      fs.satisfyStoragePolicy(file);
+
+      // Check xAttr for parent directory
+      FSNamesystem namesystem = cluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode("/file");
+      XAttrFeature f = inode.getXAttrFeature();
+      assertTrue("SPS xAttr should be exist",
+          f.getXAttr(XATTR_SATISFY_STORAGE_POLICY) != null);
+
+      cluster.restartDataNode(stopDataNode, false);
+
+      // wait and check all the file block moved in ARCHIVE
+      DFSTestUtil.waitExpectedStorageType("/file", StorageType.ARCHIVE, 3,
+          30000, cluster.getFileSystem());
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
+          return !existingXAttrs.contains(XATTR_SATISFY_STORAGE_POLICY);
+        }
+      }, 100, 10000);
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
    * Restart the hole env and trigger the DataNode's heart beats.
    * @throws Exception
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b58239d3/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 2536834..3375590 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
@@ -119,8 +119,6 @@ public class TestStoragePolicySatisfier {
   private void doTestWhenStoragePolicySetToCOLD() throws Exception {
     // Change policy to COLD
     dfs.setStoragePolicy(new Path(file), COLD);
-    FSNamesystem namesystem = hdfsCluster.getNamesystem();
-    INode inode = namesystem.getFSDirectory().getINode(file);
 
     StorageType[][] newtypes =
         new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
@@ -129,7 +127,7 @@ public class TestStoragePolicySatisfier {
     startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
         storagesPerDatanode, capacity, hdfsCluster);
 
-    namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+    dfs.satisfyStoragePolicy(new Path(file));
 
     hdfsCluster.triggerHeartbeats();
     // Wait till namenode notified about the block location details
@@ -144,8 +142,6 @@ public class TestStoragePolicySatisfier {
       createCluster();
       // Change policy to ALL_SSD
       dfs.setStoragePolicy(new Path(file), "ALL_SSD");
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.SSD, StorageType.DISK},
@@ -156,7 +152,7 @@ public class TestStoragePolicySatisfier {
       // datanodes.
       startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
           storagesPerDatanode, capacity, hdfsCluster);
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
@@ -174,8 +170,6 @@ public class TestStoragePolicySatisfier {
       createCluster();
       // Change policy to ONE_SSD
       dfs.setStoragePolicy(new Path(file), ONE_SSD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
@@ -184,7 +178,7 @@ public class TestStoragePolicySatisfier {
       // datanodes.
       startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
           storagesPerDatanode, capacity, hdfsCluster);
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
@@ -207,8 +201,6 @@ public class TestStoragePolicySatisfier {
       createCluster();
       // Change policy to ONE_SSD
       dfs.setStoragePolicy(new Path(file), ONE_SSD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
@@ -217,7 +209,7 @@ public class TestStoragePolicySatisfier {
       // datanodes.
       startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
           storagesPerDatanode, capacity, hdfsCluster);
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
 
       // Wait till the block is moved to SSD areas
@@ -250,13 +242,10 @@ public class TestStoragePolicySatisfier {
         files.add(file1);
         writeContent(file1);
       }
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      List<Long> blockCollectionIds = new ArrayList<>();
       // Change policy to ONE_SSD
       for (String fileName : files) {
         dfs.setStoragePolicy(new Path(fileName), ONE_SSD);
-        INode inode = namesystem.getFSDirectory().getINode(fileName);
-        blockCollectionIds.add(inode.getId());
+        dfs.satisfyStoragePolicy(new Path(fileName));
       }
 
       StorageType[][] newtypes =
@@ -266,9 +255,6 @@ public class TestStoragePolicySatisfier {
       // datanodes.
       startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
           storagesPerDatanode, capacity, hdfsCluster);
-      for (long inodeId : blockCollectionIds) {
-        namesystem.getBlockManager().satisfyStoragePolicy(inodeId);
-      }
       hdfsCluster.triggerHeartbeats();
 
       for (String fileName : files) {
@@ -279,7 +265,7 @@ public class TestStoragePolicySatisfier {
             fileName, StorageType.DISK, 2, 30000, dfs);
       }
 
-      waitForBlocksMovementResult(blockCollectionIds.size(), 30000);
+      waitForBlocksMovementResult(files.size(), 30000);
     } finally {
       shutdownCluster();
     }
@@ -441,8 +427,6 @@ public class TestStoragePolicySatisfier {
       createCluster();
       // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), COLD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE}};
@@ -451,7 +435,7 @@ public class TestStoragePolicySatisfier {
       startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
           storagesPerDatanode, capacity, hdfsCluster);
 
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier identified that block to move to
       // ARCHIVE area.
@@ -486,8 +470,6 @@ public class TestStoragePolicySatisfier {
       createCluster();
       // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), COLD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.DISK, StorageType.DISK}};
@@ -495,7 +477,7 @@ public class TestStoragePolicySatisfier {
       startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
           storagesPerDatanode, capacity, hdfsCluster);
 
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
 
       // No block movement will be scheduled as there is no target node
@@ -600,47 +582,51 @@ 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},
-                {StorageType.DISK, StorageType.DISK},
-                {StorageType.DISK, StorageType.DISK}})
-        .build();
-
-    hdfsCluster.waitActive();
-    dfs = hdfsCluster.getFileSystem();
+    try{
+      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},
+                  {StorageType.DISK, StorageType.DISK},
+                  {StorageType.DISK, StorageType.DISK}})
+          .build();
 
-    // create a file with replication factor 3 and mark 2 pinned block
-    // locations.
-    final String file1 = createFileAndSimulateFavoredNodes(2);
+      hdfsCluster.waitActive();
+      dfs = hdfsCluster.getFileSystem();
 
-    // Change policy to COLD
-    dfs.setStoragePolicy(new Path(file1), COLD);
-    FSNamesystem namesystem = hdfsCluster.getNamesystem();
-    INode inode = namesystem.getFSDirectory().getINode(file1);
+      // create a file with replication factor 3 and mark 2 pinned block
+      // locations.
+      final String file1 = createFileAndSimulateFavoredNodes(2);
 
-    StorageType[][] newtypes =
-        new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
-            {StorageType.ARCHIVE, StorageType.ARCHIVE},
-            {StorageType.ARCHIVE, StorageType.ARCHIVE}};
-    // Adding DISK based datanodes
-    startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
-        storagesPerDatanode, capacity, hdfsCluster);
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file1), COLD);
 
-    namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
-    hdfsCluster.triggerHeartbeats();
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+      // Adding DISK based datanodes
+      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
 
-    // No block movement will be scheduled as there is no target node available
-    // with the required storage type.
-    waitForAttemptedItems(1, 30000);
-    waitForBlocksMovementResult(1, 30000);
-    DFSTestUtil.waitExpectedStorageType(
-        file1, StorageType.ARCHIVE, 1, 30000, dfs);
-    DFSTestUtil.waitExpectedStorageType(
-        file1, StorageType.DISK, 2, 30000, dfs);
+      dfs.satisfyStoragePolicy(new Path(file1));
+      hdfsCluster.triggerHeartbeats();
+
+      // No block movement will be scheduled as there is no target node
+      // available with the required storage type.
+      waitForAttemptedItems(1, 30000);
+      waitForBlocksMovementResult(1, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file1, StorageType.ARCHIVE, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file1, StorageType.DISK, 2, 30000, dfs);
+    } finally {
+      if (hdfsCluster != null) {
+        hdfsCluster.shutdown();
+      }
+    }
   }
 
   /**
@@ -682,10 +668,8 @@ public class TestStoragePolicySatisfier {
 
       // Change policy to COLD
       dfs.setStoragePolicy(new Path(file), COLD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
 
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier identified that block to move to
       // ARCHIVE area.
@@ -723,10 +707,8 @@ public class TestStoragePolicySatisfier {
 
       // Change policy to ONE_SSD
       dfs.setStoragePolicy(new Path(file), ONE_SSD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
 
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
       DFSTestUtil.waitExpectedStorageType(
           file, StorageType.SSD, 1, 30000, dfs);
@@ -764,10 +746,7 @@ public class TestStoragePolicySatisfier {
 
       // Change policy to WARM
       dfs.setStoragePolicy(new Path(file), "WARM");
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
-
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
       hdfsCluster.triggerHeartbeats();
 
       DFSTestUtil.waitExpectedStorageType(
@@ -848,8 +827,6 @@ public class TestStoragePolicySatisfier {
 
       // Change policy to ONE_SSD
       dfs.setStoragePolicy(new Path(file), ONE_SSD);
-      FSNamesystem namesystem = hdfsCluster.getNamesystem();
-      INode inode = namesystem.getFSDirectory().getINode(file);
       Path filePath = new Path("/testChooseInSameDatanode");
       final FSDataOutputStream out =
           dfs.create(filePath, false, 100, (short) 1, 2 * DEFAULT_BLOCK_SIZE);
@@ -872,7 +849,7 @@ public class TestStoragePolicySatisfier {
       for (DataNode dataNode : dataNodes) {
         DataNodeTestUtils.setHeartbeatsDisabledForTests(dataNode, true);
       }
-      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      dfs.satisfyStoragePolicy(new Path(file));
 
       // Wait for items to be processed
       waitForAttemptedItems(1, 30000);


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


[37/50] [abbrv] hadoop git commit: HDFS-12146. [SPS]: Fix TestStoragePolicySatisfierWithStripedFile#testSPSWhenFileHasLowRedundancyBlocks. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-12146. [SPS]: Fix TestStoragePolicySatisfierWithStripedFile#testSPSWhenFileHasLowRedundancyBlocks. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 5eed1cebbf6d4d1e85ab94163570a36fa070f35e
Parents: 4d174a2
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Mon Jul 17 22:40:03 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:34 2017 +0530

----------------------------------------------------------------------
 .../server/namenode/TestStoragePolicySatisfier.java |  9 +++++----
 .../TestStoragePolicySatisfierWithStripedFile.java  | 16 ++++++++--------
 2 files changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eed1ceb/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 be7236b..10ceae7 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
@@ -1025,12 +1025,13 @@ public class TestStoragePolicySatisfier {
       list.add(cluster.stopDataNode(0));
       list.add(cluster.stopDataNode(0));
       cluster.restartNameNodes();
-      cluster.restartDataNode(list.get(0), true);
-      cluster.restartDataNode(list.get(1), true);
+      cluster.restartDataNode(list.get(0), false);
+      cluster.restartDataNode(list.get(1), false);
       cluster.waitActive();
       fs.satisfyStoragePolicy(filePath);
-      Thread.sleep(3000 * 6);
-      cluster.restartDataNode(list.get(2), true);
+      DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+          StorageType.ARCHIVE, 2, 30000, cluster.getFileSystem());
+      cluster.restartDataNode(list.get(2), false);
       DFSTestUtil.waitExpectedStorageType(filePath.toString(),
           StorageType.ARCHIVE, 3, 30000, cluster.getFileSystem());
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eed1ceb/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 f905ead..c070113 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
@@ -308,8 +308,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
    */
   @Test(timeout = 300000)
   public void testSPSWhenFileHasLowRedundancyBlocks() throws Exception {
-    // start 10 datanodes
-    int numOfDatanodes = 10;
+    // start 9 datanodes
+    int numOfDatanodes = 9;
     int storagesPerDatanode = 2;
     long capacity = 20 * defaultStripeBlockSize;
     long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
@@ -338,7 +338,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
             {StorageType.DISK, StorageType.ARCHIVE},
             {StorageType.DISK, StorageType.ARCHIVE},
             {StorageType.DISK, StorageType.ARCHIVE},
-            {StorageType.DISK, StorageType.ARCHIVE},
             {StorageType.DISK, StorageType.ARCHIVE}})
         .storageCapacities(capacities)
         .build();
@@ -366,15 +365,16 @@ public class TestStoragePolicySatisfierWithStripedFile {
       }
       cluster.restartNameNodes();
       // Restart half datanodes
-      for (int i = 0; i < numOfDatanodes / 2; i++) {
-        cluster.restartDataNode(list.get(i), true);
+      for (int i = 0; i < 5; i++) {
+        cluster.restartDataNode(list.get(i), false);
       }
       cluster.waitActive();
       fs.satisfyStoragePolicy(fooFile);
-      Thread.sleep(3000 * 6);
+      DFSTestUtil.waitExpectedStorageType(fooFile.toString(),
+          StorageType.ARCHIVE, 5, 30000, cluster.getFileSystem());
       //Start reaming datanodes
-      for (int i = numOfDatanodes - 1; i > numOfDatanodes / 2; i--) {
-        cluster.restartDataNode(list.get(i), true);
+      for (int i = numOfDatanodes - 1; i >= 5; i--) {
+        cluster.restartDataNode(list.get(i), false);
       }
       // verify storage types and locations.
       waitExpectedStorageType(cluster, fooFile.toString(), fileLen,


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


[07/50] [abbrv] hadoop git commit: HDFS-10954. [SPS]: Provide mechanism to send blocks movement result back to NN from coordinator DN. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-10954. [SPS]: Provide mechanism to send blocks movement result back to NN from coordinator DN. 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/83badc73
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/83badc73
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/83badc73

Branch: refs/heads/HDFS-10285
Commit: 83badc73c1a88c4763625156c806feacfc96a542
Parents: 7f9eb14
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Thu Nov 3 09:39:14 2016 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:20:12 2017 +0530

----------------------------------------------------------------------
 .../DatanodeProtocolClientSideTranslatorPB.java |  9 ++-
 .../DatanodeProtocolServerSideTranslatorPB.java |  4 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 52 ++++++++++++++
 .../server/blockmanagement/BlockManager.java    |  4 ++
 .../hdfs/server/datanode/BPServiceActor.java    | 25 ++++++-
 .../datanode/StoragePolicySatisfyWorker.java    | 75 ++++++++++++++++----
 .../hdfs/server/namenode/FSNamesystem.java      | 10 ++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  7 +-
 .../server/namenode/StoragePolicySatisfier.java | 23 ++++++
 .../protocol/BlocksStorageMovementResult.java   | 64 +++++++++++++++++
 .../hdfs/server/protocol/DatanodeProtocol.java  |  5 +-
 .../src/main/proto/DatanodeProtocol.proto       | 14 ++++
 .../TestNameNodePrunesMissingStorages.java      |  3 +-
 .../datanode/InternalDataNodeTestUtils.java     |  4 +-
 .../server/datanode/TestBPOfferService.java     |  8 ++-
 .../hdfs/server/datanode/TestBlockRecovery.java |  4 +-
 .../server/datanode/TestDataNodeLifeline.java   |  7 +-
 .../TestDatanodeProtocolRetryPolicy.java        |  4 +-
 .../server/datanode/TestFsDatasetCache.java     |  4 +-
 .../TestStoragePolicySatisfyWorker.java         | 13 ++--
 .../hdfs/server/datanode/TestStorageReport.java |  4 +-
 .../server/namenode/NNThroughputBenchmark.java  |  9 +--
 .../hdfs/server/namenode/NameNodeAdapter.java   |  4 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |  5 +-
 .../namenode/TestStoragePolicySatisfier.java    | 50 +++++++++++++
 25 files changed, 368 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 9cc4516..9dd87d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlock
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -138,7 +139,8 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BlocksStorageMovementResult[] blksMovementResults) throws IOException {
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
@@ -161,6 +163,11 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     if (slowDisks.haveSlowDisks()) {
       builder.addAllSlowDisks(PBHelper.convertSlowDiskInfo(slowDisks));
     }
+
+    // Adding blocks movement results to the heart beat request.
+    builder.addAllBlksMovementResults(
+        PBHelper.convertBlksMovResults(blksMovementResults));
+
     HeartbeatResponseProto resp;
     try {
       resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 5cba284..40458ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -122,7 +122,9 @@ public class DatanodeProtocolServerSideTranslatorPB implements
           request.getXceiverCount(), request.getFailedVolumes(),
           volumeFailureSummary, request.getRequestFullBlockReportLease(),
           PBHelper.convertSlowPeerInfo(request.getSlowPeersList()),
-          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()));
+          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()),
+          PBHelper.convertBlksMovResults(
+              request.getBlksMovementResultsList()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 83f3454..156c9c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailur
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockStorageMovementCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockStorageMovementProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlocksStorageMovementResultProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstructionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -98,6 +99,8 @@ import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStr
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult.Status;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
@@ -952,6 +955,55 @@ public class PBHelper {
     return SlowDiskReports.create(slowDisksMap);
   }
 
+  public static BlocksStorageMovementResult[] convertBlksMovResults(
+      List<BlocksStorageMovementResultProto> protos) {
+    BlocksStorageMovementResult[] results =
+        new BlocksStorageMovementResult[protos.size()];
+    for (int i = 0; i < protos.size(); i++) {
+      BlocksStorageMovementResultProto resultProto = protos.get(i);
+      BlocksStorageMovementResult.Status status;
+      switch (resultProto.getStatus()) {
+      case SUCCESS:
+        status = Status.SUCCESS;
+        break;
+      case FAILURE:
+        status = Status.FAILURE;
+        break;
+      default:
+        throw new AssertionError("Unknown status: " + resultProto.getStatus());
+      }
+      results[i] = new BlocksStorageMovementResult(resultProto.getTrackID(),
+          status);
+    }
+    return results;
+  }
+
+  public static List<BlocksStorageMovementResultProto> convertBlksMovResults(
+      BlocksStorageMovementResult[] blocksMovementResults) {
+    List<BlocksStorageMovementResultProto> blocksMovementResultsProto =
+        new ArrayList<>();
+    BlocksStorageMovementResultProto.Builder builder =
+        BlocksStorageMovementResultProto.newBuilder();
+    for (int i = 0; i < blocksMovementResults.length; i++) {
+      BlocksStorageMovementResult report = blocksMovementResults[i];
+      builder.setTrackID(report.getTrackId());
+      BlocksStorageMovementResultProto.Status status;
+      switch (report.getStatus()) {
+      case SUCCESS:
+        status = BlocksStorageMovementResultProto.Status.SUCCESS;
+        break;
+      case FAILURE:
+        status = BlocksStorageMovementResultProto.Status.FAILURE;
+        break;
+      default:
+        throw new AssertionError("Unknown status: " + report.getStatus());
+      }
+      builder.setStatus(status);
+      blocksMovementResultsProto.add(builder.build());
+    }
+    return blocksMovementResultsProto;
+  }
+
   public static JournalInfo convert(JournalInfoProto info) {
     int lv = info.hasLayoutVersion() ? info.getLayoutVersion() : 0;
     int nsID = info.hasNamespaceID() ? info.getNamespaceID() : 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/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 3ae7b58..8487a4a 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
@@ -4922,4 +4922,8 @@ public class BlockManager implements BlockStatsMXBean {
   public void satisfyStoragePolicy(long id) {
     storageMovementNeeded.add(id);
   }
+
+  public StoragePolicySatisfier getStoragePolicySatisfier() {
+    return sps;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index a94d2df..0f93fb0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -511,6 +512,10 @@ class BPServiceActor implements Runnable {
         outliersReportDue && dn.getDiskMetrics() != null ?
             SlowDiskReports.create(dn.getDiskMetrics().getDiskOutliersStats()) :
             SlowDiskReports.EMPTY_REPORT;
+
+    BlocksStorageMovementResult[] blksMovementResults =
+        getBlocksMovementResults();
+
     HeartbeatResponse response = bpNamenode.sendHeartbeat(bpRegistration,
         reports,
         dn.getFSDataset().getCacheCapacity(),
@@ -521,15 +526,33 @@ class BPServiceActor implements Runnable {
         volumeFailureSummary,
         requestBlockReportLease,
         slowPeers,
-        slowDisks);
+        slowDisks,
+        blksMovementResults);
 
     if (outliersReportDue) {
       // If the report was due and successfully sent, schedule the next one.
       scheduler.scheduleNextOutlierReport();
     }
+
+    // Remove the blocks movement results after successfully transferring
+    // to namenode.
+    dn.getStoragePolicySatisfyWorker().getBlocksMovementsCompletionHandler()
+        .remove(blksMovementResults);
+
     return response;
   }
 
+  private BlocksStorageMovementResult[] getBlocksMovementResults() {
+    List<BlocksStorageMovementResult> trackIdVsMovementStatus = dn
+        .getStoragePolicySatisfyWorker().getBlocksMovementsCompletionHandler()
+        .getBlksMovementResults();
+    BlocksStorageMovementResult[] blksMovementResult =
+        new BlocksStorageMovementResult[trackIdVsMovementStatus.size()];
+    trackIdVsMovementStatus.toArray(blksMovementResult);
+
+    return blksMovementResult;
+  }
+
   @VisibleForTesting
   void sendLifelineForTests() throws IOException {
     lifelineSender.sendLifeline();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 604fb4a..1bd851e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -29,6 +29,7 @@ import java.io.OutputStream;
 import java.net.Socket;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -57,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
@@ -218,7 +220,8 @@ public class StoragePolicySatisfyWorker {
         OutputStream unbufOut = sock.getOutputStream();
         InputStream unbufIn = sock.getInputStream();
         Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
-            extendedBlock, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
+            extendedBlock, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
+            new StorageType[]{targetStorageType}, new String[0]);
 
         DataEncryptionKeyFactory keyFactory = datanode
             .getDataEncryptionKeyFactoryForBlock(extendedBlock);
@@ -257,7 +260,7 @@ public class StoragePolicySatisfyWorker {
         Token<BlockTokenIdentifier> accessToken, DatanodeInfo srcDn,
         StorageType destinStorageType) throws IOException {
       new Sender(out).replaceBlock(eb, destinStorageType, accessToken,
-          srcDn.getDatanodeUuid(), srcDn);
+          srcDn.getDatanodeUuid(), srcDn, null);
     }
 
     /** Receive a reportedBlock copy response from the input stream. */
@@ -276,7 +279,7 @@ public class StoragePolicySatisfyWorker {
   /**
    * Block movement status code.
    */
-  enum BlockMovementStatus {
+  public static enum BlockMovementStatus {
     /** Success. */
     DN_BLK_STORAGE_MOVEMENT_SUCCESS(0),
     /**
@@ -343,26 +346,72 @@ public class StoragePolicySatisfyWorker {
 
   /**
    * Blocks movements completion handler, which is used to collect details of
-   * the completed list of block movements and notify the namenode about the
-   * success or failures.
+   * the completed list of block movements and this status(success or failure)
+   * will be send to the namenode via heartbeat.
    */
   static class BlocksMovementsCompletionHandler {
-    private final List<BlockMovementResult> completedBlocks = new ArrayList<>();
+    private final List<BlocksStorageMovementResult> trackIdVsMovementStatus =
+        new ArrayList<>();
 
     /**
-     * Collect all the block movement results and notify namenode.
+     * Collect all the block movement results. Later this will be send to
+     * namenode via heart beat.
      *
      * @param results
      *          result of all the block movements per trackId
      */
-    void handle(List<BlockMovementResult> results) {
-      completedBlocks.addAll(results);
-      // TODO: notify namenode about the success/failures.
+    void handle(List<BlockMovementResult> resultsPerTrackId) {
+      BlocksStorageMovementResult.Status status =
+          BlocksStorageMovementResult.Status.SUCCESS;
+      long trackId = -1;
+      for (BlockMovementResult blockMovementResult : resultsPerTrackId) {
+        trackId = blockMovementResult.getTrackId();
+        if (blockMovementResult.status ==
+            BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE) {
+          status = BlocksStorageMovementResult.Status.FAILURE;
+          // If any of the block movement is failed, then mark as failure so
+          // that namenode can take a decision to retry the blocks associated to
+          // the given trackId.
+          break;
+        }
+      }
+
+      // Adding to the tracking results list. Later this will be send to
+      // namenode via datanode heartbeat.
+      synchronized (trackIdVsMovementStatus) {
+        trackIdVsMovementStatus.add(
+            new BlocksStorageMovementResult(trackId, status));
+      }
+    }
+
+    /**
+     * @return unmodifiable list of blocks storage movement results.
+     */
+    List<BlocksStorageMovementResult> getBlksMovementResults() {
+      synchronized (trackIdVsMovementStatus) {
+        if (trackIdVsMovementStatus.size() <= 0) {
+          return new ArrayList<>();
+        }
+        List<BlocksStorageMovementResult> results = Collections
+            .unmodifiableList(trackIdVsMovementStatus);
+        return results;
+      }
     }
 
-    @VisibleForTesting
-    List<BlockMovementResult> getCompletedBlocks() {
-      return completedBlocks;
+    /**
+     * Remove the blocks storage movement results.
+     *
+     * @param results
+     *          set of blocks storage movement results
+     */
+    void remove(BlocksStorageMovementResult[] results) {
+      if (results != null) {
+        synchronized (trackIdVsMovementStatus) {
+          for (BlocksStorageMovementResult blocksMovementResult : results) {
+            trackIdVsMovementStatus.remove(blocksMovementResult);
+          }
+        }
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/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 6a890e2..c524a91 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
@@ -263,6 +263,7 @@ import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
 import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics;
 import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -3755,7 +3756,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BlocksStorageMovementResult[] blksMovementResults) throws IOException {
     readLock();
     try {
       //get datanode commands
@@ -3769,6 +3771,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (requestFullBlockReportLease) {
         blockReportLeaseId =  blockManager.requestBlockReportLeaseId(nodeReg);
       }
+
+      // TODO: Handle blocks movement results send by the coordinator datanode.
+      // This has to be revisited as part of HDFS-11029.
+      blockManager.getStoragePolicySatisfier()
+          .handleBlocksStorageMovementResults(blksMovementResults);
+
       //create ha status
       final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(
           haContext.getState().getServiceState(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 36d33a6..3b325d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -152,6 +152,7 @@ import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -1461,13 +1462,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       int failedVolumes, VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BlocksStorageMovementResult[] blkMovementStatus) throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
         dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
         failedVolumes, volumeFailureSummary, requestFullBlockReportLease,
-        slowPeers, slowDisks);
+        slowPeers, slowDisks,
+        blkMovementStatus);
   }
 
   @Override // DatanodeProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/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 b5aed37..fbe686a 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
@@ -39,11 +39,14 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Setting storagePolicy on a file after the file write will only update the new
  * storage policy type in Namespace, but physical block storage movement will
@@ -394,4 +397,24 @@ public class StoragePolicySatisfier implements Runnable {
       return typeNodeMap.get(type);
     }
   }
+
+  // TODO: Temporarily keeping the results for assertion. This has to be
+  // revisited as part of HDFS-11029.
+  @VisibleForTesting
+  List<BlocksStorageMovementResult> results = new ArrayList<>();
+
+  /**
+   * Receives the movement results of collection of blocks associated to a
+   * trackId.
+   *
+   * @param blksMovementResults
+   *          movement status of the set of blocks associated to a trackId.
+   */
+  void handleBlocksStorageMovementResults(
+      BlocksStorageMovementResult[] blksMovementResults) {
+    if (blksMovementResults.length <= 0) {
+      return;
+    }
+    results.addAll(Arrays.asList(blksMovementResults));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/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
new file mode 100644
index 0000000..1afba34
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksStorageMovementResult.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+/**
+ * This class represents, movement status of a set of blocks associated to a
+ * track Id.
+ */
+public class BlocksStorageMovementResult {
+
+  private final long trackId;
+  private final Status status;
+
+  /**
+   * SUCCESS - If all the blocks associated to track id has moved successfully
+   * or maximum possible movements done.
+   *
+   * <p>
+   * FAILURE - If any of its(trackId) blocks movement failed and requires to
+   * retry these failed blocks movements. Example selected target node is no
+   * more running or no space. So, retrying by selecting new target node might
+   * work.
+   */
+  public static enum Status {
+    SUCCESS, FAILURE;
+  }
+
+  /**
+   * BlocksStorageMovementResult constructor.
+   *
+   * @param trackId
+   *          tracking identifier
+   * @param status
+   *          block movement status
+   */
+  public BlocksStorageMovementResult(long trackId, Status status) {
+    this.trackId = trackId;
+    this.status = status;
+  }
+
+  public long getTrackId() {
+    return trackId;
+  }
+
+  public Status getStatus() {
+    return status;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 283f367..858f59b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -111,6 +111,8 @@ public interface DatanodeProtocol {
    * @param slowPeers Details of peer DataNodes that were detected as being
    *                  slow to respond to packet writes. Empty report if no
    *                  slow peers were detected by the DataNode.
+   * @param blksMovementResults array of movement status of a set of blocks
+   *                            associated to a trackId.
    * @throws IOException on error
    */
   @Idempotent
@@ -124,7 +126,8 @@ public interface DatanodeProtocol {
                                        VolumeFailureSummary volumeFailureSummary,
                                        boolean requestFullBlockReportLease,
                                        @Nonnull SlowPeerReports slowPeers,
-                                       @Nonnull SlowDiskReports slowDisks)
+                                       @Nonnull SlowDiskReports slowDisks,
+                                       BlocksStorageMovementResult[] blksMovementResults)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 8e19809..77b0f86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -177,6 +177,18 @@ message BlockStorageMovementProto {
 }
 
 /**
+ * Movement status of the set of blocks associated to a trackId.
+ */
+message BlocksStorageMovementResultProto {
+  enum Status {
+    SUCCESS = 1; // block movement succeeded
+    FAILURE = 2; // block movement failed and needs to retry
+  }
+  required uint64 trackID = 1;
+  required Status status = 2;
+}
+
+/**
  * registration - Information of the datanode registering with the namenode
  */
 message RegisterDatanodeRequestProto {
@@ -219,6 +231,7 @@ message VolumeFailureSummaryProto {
  * volumeFailureSummary - info about volume failures
  * slowPeers - info about peer DataNodes that are suspected to be slow.
  * slowDisks - info about DataNode disks that are suspected to be slow.
+ * blksMovementResults - status of the scheduled blocks movements
  */
 message HeartbeatRequestProto {
   required DatanodeRegistrationProto registration = 1; // Datanode info
@@ -232,6 +245,7 @@ message HeartbeatRequestProto {
   optional bool requestFullBlockReportLease = 9 [ default = false ];
   repeated SlowPeerReportProto slowPeers = 10;
   repeated SlowDiskReportProto slowDisks = 11;
+  repeated BlocksStorageMovementResultProto blksMovementResults = 12;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 948a8fb..2d58732 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
@@ -116,7 +117,7 @@ public class TestNameNodePrunesMissingStorages {
       cluster.stopDataNode(0);
       cluster.getNameNodeRpc().sendHeartbeat(dnReg, prunedReports, 0L, 0L, 0, 0,
           0, null, true, SlowPeerReports.EMPTY_REPORT,
-          SlowDiskReports.EMPTY_REPORT);
+          SlowDiskReports.EMPTY_REPORT, new BlocksStorageMovementResult[0]);
 
       // Check that the missing storage was pruned.
       assertThat(dnDescriptor.getStorageInfos().length, is(expectedStoragesAfterTest));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
index b453991..bd831d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
@@ -167,7 +168,8 @@ public class InternalDataNodeTestUtils {
             Mockito.anyInt(), Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class))).thenReturn(
+            Mockito.any(SlowDiskReports.class),
+            Mockito.any(BlocksStorageMovementResult[].class))).thenReturn(
         new HeartbeatResponse(new DatanodeCommand[0], new NNHAStatusHeartbeat(
             HAServiceState.ACTIVE, 1), null, ThreadLocalRandom.current()
             .nextLong() | 1L));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index ec19926..524b8b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -122,6 +123,8 @@ public class TestBPOfferService {
     Mockito.doReturn(new DNConf(mockDn)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn"))
         .when(mockDn).getMetrics();
+    Mockito.doReturn(new StoragePolicySatisfyWorker(conf, mockDn)).when(mockDn)
+        .getStoragePolicySatisfyWorker();
 
     // Set up a simulated dataset with our fake BP
     mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, conf));
@@ -156,7 +159,8 @@ public class TestBPOfferService {
           Mockito.any(VolumeFailureSummary.class),
           Mockito.anyBoolean(),
           Mockito.any(SlowPeerReports.class),
-          Mockito.any(SlowDiskReports.class));
+          Mockito.any(SlowDiskReports.class),
+          Mockito.any(BlocksStorageMovementResult[].class));
     mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     datanodeCommands[nnIdx] = new DatanodeCommand[0];
     return mock;
@@ -375,6 +379,8 @@ public class TestBPOfferService {
     Mockito.doReturn(new DNConf(mockDn)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn")).
       when(mockDn).getMetrics();
+    Mockito.doReturn(new StoragePolicySatisfyWorker(conf, mockDn)).when(mockDn)
+        .getStoragePolicySatisfyWorker();
     final AtomicInteger count = new AtomicInteger();
     Mockito.doAnswer(new Answer<Void>() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index e9bd7a8..38372ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -87,6 +87,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -226,7 +227,8 @@ public class TestBlockRecovery {
             Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class)))
+            Mockito.any(SlowDiskReports.class),
+            Mockito.any(BlocksStorageMovementResult[].class)))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
             new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
index 28427bc..b15b530 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
@@ -172,7 +173,8 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class));
+            any(SlowDiskReports.class),
+            any(BlocksStorageMovementResult[].class));
 
     // Intercept lifeline to trigger latch count-down on each call.
     doAnswer(new LatchCountingAnswer<Void>(lifelinesSent))
@@ -237,7 +239,8 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class));
+            any(SlowDiskReports.class),
+            any(BlocksStorageMovementResult[].class));
 
     // While waiting on the latch for the expected number of heartbeat messages,
     // poll DataNode tracking information.  We expect that the DataNode always

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
index bb1d9ef..d7ac3f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
@@ -222,7 +223,8 @@ public class TestDatanodeProtocolRetryPolicy {
            Mockito.any(VolumeFailureSummary.class),
            Mockito.anyBoolean(),
            Mockito.any(SlowPeerReports.class),
-           Mockito.any(SlowDiskReports.class));
+           Mockito.any(SlowDiskReports.class),
+           Mockito.any(BlocksStorageMovementResult[].class));
 
     dn = new DataNode(conf, locations, null, null) {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
index 2dbd5b9..b9f21a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.Page
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -208,7 +209,8 @@ public class TestFsDatasetCache {
           (StorageReport[]) any(), anyLong(), anyLong(),
           anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
           anyBoolean(), any(SlowPeerReports.class),
-          any(SlowDiskReports.class));
+          any(SlowDiskReports.class),
+          (BlocksStorageMovementResult[]) any());
     } finally {
       lock.writeLock().unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/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 ea3eec3..1eb44e0 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
@@ -34,10 +34,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementResult;
-import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -191,12 +190,12 @@ public class TestStoragePolicySatisfyWorker {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        List<BlockMovementResult> completedBlocks = worker
-            .getBlocksMovementsCompletionHandler().getCompletedBlocks();
+        List<BlocksStorageMovementResult> completedBlocks = worker
+            .getBlocksMovementsCompletionHandler().getBlksMovementResults();
         int failedCount = 0;
-        for (BlockMovementResult blockMovementResult : completedBlocks) {
-          if (BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE ==
-              blockMovementResult.getStatus()) {
+        for (BlocksStorageMovementResult blkMovementResult : completedBlocks) {
+          if (blkMovementResult.getStatus() ==
+              BlocksStorageMovementResult.Status.FAILURE) {
             failedCount++;
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
index 5f62ddb..df120ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
@@ -110,7 +111,8 @@ public class TestStorageReport {
         anyLong(), anyLong(), anyInt(), anyInt(), anyInt(),
         Mockito.any(VolumeFailureSummary.class), Mockito.anyBoolean(),
         Mockito.any(SlowPeerReports.class),
-        Mockito.any(SlowDiskReports.class));
+        Mockito.any(SlowDiskReports.class),
+        Mockito.any(BlocksStorageMovementResult[].class));
 
     StorageReport[] reports = captor.getValue();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 3a3c471..1e016f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -956,8 +957,8 @@ public class NNThroughputBenchmark implements Tool {
           DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0L) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration, rep,
           0L, 0L, 0, 0, 0, null, true,
-          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-          .getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+          new BlocksStorageMovementResult[0]).getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -1007,8 +1008,8 @@ public class NNThroughputBenchmark implements Tool {
           false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration,
           rep, 0L, 0L, 0, 0, 0, null, true,
-          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-          .getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+          new BlocksStorageMovementResult[0]).getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index 242e8f5..ba29c82 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
@@ -126,7 +127,8 @@ public class NameNodeAdapter {
     return namesystem.handleHeartbeat(nodeReg,
         BlockManagerTestUtil.getStorageReportsForDatanode(dd),
         dd.getCacheCapacity(), dd.getCacheRemaining(), 0, 0, 0, null, true,
-        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
+        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+        new BlocksStorageMovementResult[0]);
   }
 
   public static boolean setReplication(final FSNamesystem ns,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 366f584..36beaa8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.InternalDataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -139,8 +140,8 @@ public class TestDeadDatanode {
         false, 0, 0, 0, 0, 0) };
     DatanodeCommand[] cmd =
         dnp.sendHeartbeat(reg, rep, 0L, 0L, 0, 0, 0, null, true,
-            SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-            .getCommands();
+            SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+            new BlocksStorageMovementResult[0]).getCommands();
     assertEquals(1, cmd.length);
     assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83badc73/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 37664b5..cbfdfc6 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -27,6 +28,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Test;
@@ -146,6 +148,54 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Tests to verify that the block storage movement results will be propagated
+   * to Namenode via datanode heartbeat.
+   */
+  @Test(timeout = 300000)
+  public void testPerTrackIdBlocksStorageMovementResults() throws Exception {
+    try {
+      // Change policy to ONE_SSD
+      distributedFS.setStoragePolicy(new Path(file), "ONE_SSD");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+
+      // Wait till the block is moved to SSD areas
+      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, distributedFS, 2, 30000);
+
+      // TODO: Temporarily using the results from StoragePolicySatisfier class.
+      // This has to be revisited as part of HDFS-11029.
+      waitForBlocksMovementResult(1, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  private void waitForBlocksMovementResult(int expectedResultsCount,
+      int timeout) throws TimeoutException, InterruptedException {
+    BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
+    final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("expectedResultsCount={} actualResultsCount={}",
+            expectedResultsCount, sps.results.size());
+        return expectedResultsCount == sps.results.size();
+      }
+    }, 100, timeout);
+  }
+
   private void writeContent(final DistributedFileSystem dfs,
       final String fileName) throws IOException {
     // write to DISK


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


[15/50] [abbrv] hadoop git commit: HDFS-11248: [SPS]: Handle partial block location movements. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-11248: [SPS]: Handle partial block location movements. 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/fe2cb59f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fe2cb59f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fe2cb59f

Branch: refs/heads/HDFS-10285
Commit: fe2cb59fae9b0a62896182b6dc25c47a84d0c444
Parents: 37a4d84
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Dec 28 23:21:07 2016 -0800
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:30 2017 +0530

----------------------------------------------------------------------
 .../datanode/BlockStorageMovementTracker.java   |  15 --
 .../datanode/StoragePolicySatisfyWorker.java    |  15 +-
 .../BlockStorageMovementAttemptedItems.java     | 206 +++++++++++++-----
 .../server/namenode/StoragePolicySatisfier.java | 215 +++++++++++++------
 .../TestBlockStorageMovementAttemptedItems.java | 101 ++++++++-
 .../namenode/TestStoragePolicySatisfier.java    |  63 +++++-
 6 files changed, 454 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe2cb59f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
index 2de88fc..bd35b09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -28,7 +28,6 @@ import java.util.concurrent.Future;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementResult;
-import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementStatus;
 import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlocksMovementsCompletionHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -109,20 +108,6 @@ public class BlockStorageMovementTracker implements Runnable {
     }
   }
 
-  /**
-   * Mark as block movement failure for the given trackId and blockId.
-   *
-   * @param trackId tracking id
-   * @param blockId block id
-   */
-  void markBlockMovementFailure(long trackId, long blockId) {
-    LOG.debug("Mark as block movement failure for the given "
-        + "trackId:{} and blockId:{}", trackId, blockId);
-    BlockMovementResult result = new BlockMovementResult(trackId, blockId, null,
-        BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE);
-    addMovementResultToTrackIdList(result);
-  }
-
   private List<BlockMovementResult> addMovementResultToTrackIdList(
       BlockMovementResult result) {
     long trackId = result.getTrackId();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe2cb59f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 19f3fe2..10adbfd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -154,18 +154,9 @@ public class StoragePolicySatisfyWorker {
       Collection<BlockMovingInfo> blockMovingInfos) {
     LOG.debug("Received BlockMovingTasks {}", blockMovingInfos);
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
-      // Iterating backwards. This is to ensure that all the block src location
-      // which doesn't have a target node will be marked as failure before
-      // scheduling the block movement to valid target nodes.
-      for (int i = blkMovingInfo.getSources().length - 1; i >= 0; i--) {
-        if (i >= blkMovingInfo.getTargets().length) {
-          // Since there is no target selected for scheduling the block,
-          // just mark this block storage movement as failure. Later, namenode
-          // can take action on this.
-          movementTracker.markBlockMovementFailure(trackID,
-              blkMovingInfo.getBlock().getBlockId());
-          continue;
-        }
+      assert blkMovingInfo.getSources().length == blkMovingInfo
+          .getTargets().length;
+      for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
         DatanodeInfo target = blkMovingInfo.getTargets()[i];
         BlockMovingTask blockMovingTask = new BlockMovingTask(
             trackID, blockPoolID, blkMovingInfo.getBlock(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe2cb59f/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 bb26082..ce97075 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
@@ -43,11 +43,14 @@ import com.google.common.annotations.VisibleForTesting;
  * automatically after timeout. The default timeout would be 30mins.
  */
 public class BlockStorageMovementAttemptedItems {
-  public static final Logger LOG =
+  private static final Logger LOG =
       LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class);
-  // A map holds the items which are already taken for blocks movements
-  // processing and sent to DNs.
-  private final Map<Long, Long> storageMovementAttemptedItems;
+
+  /**
+   * A map holds the items which are already taken for blocks movements
+   * processing and sent to DNs.
+   */
+  private final Map<Long, ItemInfo> storageMovementAttemptedItems;
   private final List<BlocksStorageMovementResult> storageMovementAttemptedResults;
   private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
@@ -83,10 +86,16 @@ public class BlockStorageMovementAttemptedItems {
    *
    * @param blockCollectionID
    *          - tracking id / block collection id
+   * @param allBlockLocsAttemptedToSatisfy
+   *          - failed to find matching target nodes to satisfy storage type for
+   *          all the block locations of the given blockCollectionID
    */
-  public void add(Long blockCollectionID) {
+  public void add(Long blockCollectionID,
+      boolean allBlockLocsAttemptedToSatisfy) {
     synchronized (storageMovementAttemptedItems) {
-      storageMovementAttemptedItems.put(blockCollectionID, monotonicNow());
+      ItemInfo itemInfo = new ItemInfo(monotonicNow(),
+          allBlockLocsAttemptedToSatisfy);
+      storageMovementAttemptedItems.put(blockCollectionID, itemInfo);
     }
   }
 
@@ -121,15 +130,62 @@ public class BlockStorageMovementAttemptedItems {
    */
   public synchronized void stop() {
     monitorRunning = false;
-    timerThread.interrupt();
-    try {
-      timerThread.join(3000);
-    } catch (InterruptedException ie) {
+    if (timerThread != null) {
+      timerThread.interrupt();
+      try {
+        timerThread.join(3000);
+      } catch (InterruptedException ie) {
+      }
     }
     this.clearQueues();
   }
 
   /**
+   * This class contains information of an attempted trackID. Information such
+   * as, (a)last attempted time stamp, (b)whether all the blocks in the trackID
+   * were attempted and blocks movement has been scheduled to satisfy storage
+   * policy. This is used by
+   * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
+   */
+  private final static class ItemInfo {
+    private final long lastAttemptedTimeStamp;
+    private final boolean allBlockLocsAttemptedToSatisfy;
+
+    /**
+     * ItemInfo constructor.
+     *
+     * @param lastAttemptedTimeStamp
+     *          last attempted time stamp
+     * @param allBlockLocsAttemptedToSatisfy
+     *          whether all the blocks in the trackID were attempted and blocks
+     *          movement has been scheduled to satisfy storage policy
+     */
+    private ItemInfo(long lastAttemptedTimeStamp,
+        boolean allBlockLocsAttemptedToSatisfy) {
+      this.lastAttemptedTimeStamp = lastAttemptedTimeStamp;
+      this.allBlockLocsAttemptedToSatisfy = allBlockLocsAttemptedToSatisfy;
+    }
+
+    /**
+     * @return last attempted time stamp.
+     */
+    private long getLastAttemptedTimeStamp() {
+      return lastAttemptedTimeStamp;
+    }
+
+    /**
+     * @return true/false. True value represents that, all the block locations
+     *         under the trackID has found matching target nodes to satisfy
+     *         storage policy. False value represents that, trackID needed
+     *         retries to satisfy the storage policy for some of the block
+     *         locations.
+     */
+    private boolean isAllBlockLocsAttemptedToSatisfy() {
+      return allBlockLocsAttemptedToSatisfy;
+    }
+  }
+
+  /**
    * A monitor class for checking block storage movement result and long waiting
    * items periodically.
    */
@@ -147,76 +203,108 @@ public class BlockStorageMovementAttemptedItems {
         }
       }
     }
+  }
 
-    private void blocksStorageMovementUnReportedItemsCheck() {
-      synchronized (storageMovementAttemptedItems) {
-        Iterator<Entry<Long, Long>> iter =
-            storageMovementAttemptedItems.entrySet().iterator();
-        long now = monotonicNow();
-        while (iter.hasNext()) {
-          Entry<Long, Long> entry = iter.next();
-          if (now > entry.getValue() + selfRetryTimeout) {
-            Long blockCollectionID = entry.getKey();
-            synchronized (storageMovementAttemptedResults) {
-              boolean exist = isExistInResult(blockCollectionID);
-              if (!exist) {
-                blockStorageMovementNeeded.add(blockCollectionID);
-              } else {
-                LOG.info("Blocks storage movement results for the"
-                    + " tracking id : " + blockCollectionID
-                    + " is reported from one of the co-ordinating datanode."
-                    + " So, the result will be processed soon.");
-              }
+  @VisibleForTesting
+  void blocksStorageMovementUnReportedItemsCheck() {
+    synchronized (storageMovementAttemptedItems) {
+      Iterator<Entry<Long, ItemInfo>> iter = storageMovementAttemptedItems
+          .entrySet().iterator();
+      long now = monotonicNow();
+      while (iter.hasNext()) {
+        Entry<Long, ItemInfo> entry = iter.next();
+        ItemInfo itemInfo = entry.getValue();
+        if (now > itemInfo.getLastAttemptedTimeStamp() + selfRetryTimeout) {
+          Long blockCollectionID = entry.getKey();
+          synchronized (storageMovementAttemptedResults) {
+            if (!isExistInResult(blockCollectionID)) {
+              blockStorageMovementNeeded.add(blockCollectionID);
               iter.remove();
+              LOG.info("TrackID: {} becomes timed out and moved to needed "
+                  + "retries queue for next iteration.", blockCollectionID);
+            } else {
+              LOG.info("Blocks storage movement results for the"
+                  + " tracking id : " + blockCollectionID
+                  + " is reported from one of the co-ordinating datanode."
+                  + " So, the result will be processed soon.");
             }
           }
         }
-
       }
+
     }
+  }
 
-    private boolean isExistInResult(Long blockCollectionID) {
-      Iterator<BlocksStorageMovementResult> iter =
-          storageMovementAttemptedResults.iterator();
-      while (iter.hasNext()) {
-        BlocksStorageMovementResult storageMovementAttemptedResult =
-            iter.next();
-        if (storageMovementAttemptedResult.getTrackId() == blockCollectionID) {
-          return true;
-        }
+  private boolean isExistInResult(Long blockCollectionID) {
+    Iterator<BlocksStorageMovementResult> iter = storageMovementAttemptedResults
+        .iterator();
+    while (iter.hasNext()) {
+      BlocksStorageMovementResult storageMovementAttemptedResult = iter.next();
+      if (storageMovementAttemptedResult.getTrackId() == blockCollectionID) {
+        return true;
       }
-      return false;
     }
+    return false;
+  }
 
-    private void blockStorageMovementResultCheck() {
-      synchronized (storageMovementAttemptedResults) {
-        Iterator<BlocksStorageMovementResult> iter =
-            storageMovementAttemptedResults.iterator();
-        while (iter.hasNext()) {
-          BlocksStorageMovementResult storageMovementAttemptedResult =
-              iter.next();
+  @VisibleForTesting
+  void blockStorageMovementResultCheck() {
+    synchronized (storageMovementAttemptedResults) {
+      Iterator<BlocksStorageMovementResult> resultsIter =
+          storageMovementAttemptedResults.iterator();
+      while (resultsIter.hasNext()) {
+        // TrackID need to be retried in the following cases:
+        // 1) All or few scheduled block(s) movement has been failed.
+        // 2) All the scheduled block(s) movement has been succeeded but there
+        // are unscheduled block(s) movement in this trackID. Say, some of
+        // the blocks in the trackID couldn't finding any matching target node
+        // for scheduling block movement in previous SPS iteration.
+        BlocksStorageMovementResult storageMovementAttemptedResult = resultsIter
+            .next();
+        synchronized (storageMovementAttemptedItems) {
           if (storageMovementAttemptedResult
               .getStatus() == BlocksStorageMovementResult.Status.FAILURE) {
             blockStorageMovementNeeded
                 .add(storageMovementAttemptedResult.getTrackId());
-            LOG.warn("Blocks storage movement results for the tracking id : "
-                + storageMovementAttemptedResult.getTrackId()
+            LOG.warn("Blocks storage movement results for the tracking id: {}"
                 + " is reported from co-ordinating datanode, but result"
-                + " status is FAILURE. So, added for retry");
+                + " status is FAILURE. So, added for retry",
+                storageMovementAttemptedResult.getTrackId());
           } else {
-            synchronized (storageMovementAttemptedItems) {
-              storageMovementAttemptedItems
-                  .remove(storageMovementAttemptedResult.getTrackId());
+            ItemInfo itemInfo = storageMovementAttemptedItems
+                .get(storageMovementAttemptedResult.getTrackId());
+
+            // ItemInfo could be null. One case is, before the blocks movements
+            // result arrives the attempted trackID became timed out and then
+            // removed the trackID from the storageMovementAttemptedItems list.
+            // TODO: Need to ensure that trackID is added to the
+            // 'blockStorageMovementNeeded' queue for retries to handle the
+            // following condition. If all the block locations under the trackID
+            // are attempted and failed to find matching target nodes to satisfy
+            // storage policy in previous SPS iteration.
+            if (itemInfo != null
+                && !itemInfo.isAllBlockLocsAttemptedToSatisfy()) {
+              blockStorageMovementNeeded
+                  .add(storageMovementAttemptedResult.getTrackId());
+              LOG.warn("Blocks storage movement is SUCCESS for the track id: {}"
+                  + " reported from co-ordinating datanode. But adding trackID"
+                  + " back to retry queue as some of the blocks couldn't find"
+                  + " matching target nodes in previous SPS iteration.",
+                  storageMovementAttemptedResult.getTrackId());
+            } else {
+              LOG.info("Blocks storage movement is SUCCESS for the track id: {}"
+                  + " reported from co-ordinating datanode. But the trackID "
+                  + "doesn't exists in storageMovementAttemptedItems list",
+                  storageMovementAttemptedResult.getTrackId());
             }
-            LOG.info("Blocks storage movement results for the tracking id : "
-                + storageMovementAttemptedResult.getTrackId()
-                + " is reported from co-ordinating datanode. "
-                + "The result status is SUCCESS.");
           }
-          iter.remove(); // remove from results as processed above
+          // Remove trackID from the attempted list, if any.
+          storageMovementAttemptedItems
+              .remove(storageMovementAttemptedResult.getTrackId());
         }
+        // Remove trackID from results as processed above.
+        resultsIter.remove();
       }
-
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe2cb59f/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 56a531f..26e0775 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
@@ -162,8 +162,15 @@ public class StoragePolicySatisfier implements Runnable {
       try {
         Long blockCollectionID = storageMovementNeeded.get();
         if (blockCollectionID != null) {
-          computeAndAssignStorageMismatchedBlocksToDNs(blockCollectionID);
-          this.storageMovementsMonitor.add(blockCollectionID);
+          BlockCollection blockCollection =
+              namesystem.getBlockCollection(blockCollectionID);
+          // Check blockCollectionId existence.
+          if (blockCollection != null) {
+            boolean allBlockLocsAttemptedToSatisfy =
+                computeAndAssignStorageMismatchedBlocksToDNs(blockCollection);
+            this.storageMovementsMonitor.add(blockCollectionID,
+                allBlockLocsAttemptedToSatisfy);
+          }
         }
         // TODO: We can think to make this as configurable later, how frequently
         // we want to check block movements.
@@ -192,20 +199,17 @@ public class StoragePolicySatisfier implements Runnable {
     }
   }
 
-  private void computeAndAssignStorageMismatchedBlocksToDNs(
-      long blockCollectionID) {
-    BlockCollection blockCollection =
-        namesystem.getBlockCollection(blockCollectionID);
-    if (blockCollection == null) {
-      return;
-    }
+  private boolean computeAndAssignStorageMismatchedBlocksToDNs(
+      BlockCollection blockCollection) {
     byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
     BlockStoragePolicy existingStoragePolicy =
         blockManager.getStoragePolicy(existingStoragePolicyID);
     if (!blockCollection.getLastBlock().isComplete()) {
       // Postpone, currently file is under construction
       // So, should we add back? or leave it to user
-      return;
+      LOG.info("BlockCollectionID: {} file is under construction. So, postpone"
+          + " this to the next retry iteration", blockCollection.getId());
+      return true;
     }
 
     // First datanode will be chosen as the co-ordinator node for storage
@@ -213,61 +217,87 @@ public class StoragePolicySatisfier implements Runnable {
     DatanodeDescriptor coordinatorNode = null;
     BlockInfo[] blocks = blockCollection.getBlocks();
     List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
+
+    // True value represents that, SPS is able to find matching target nodes
+    // to satisfy storage type for all the blocks locations of the given
+    // blockCollection. A false value represents that, blockCollection needed
+    // retries to satisfy the storage policy for some of the block locations.
+    boolean foundMatchingTargetNodesForAllBlocks = true;
+
     for (int i = 0; i < blocks.length; i++) {
       BlockInfo blockInfo = blocks[i];
-      List<StorageType> expectedStorageTypes =
-          existingStoragePolicy.chooseStorageTypes(blockInfo.getReplication());
-      DatanodeStorageInfo[] storages = blockManager.getStorages(blockInfo);
-      StorageType[] storageTypes = new StorageType[storages.length];
-      for (int j = 0; j < storages.length; j++) {
-        DatanodeStorageInfo datanodeStorageInfo = storages[j];
-        StorageType storageType = datanodeStorageInfo.getStorageType();
-        storageTypes[j] = storageType;
-      }
-      List<StorageType> existing =
-          new LinkedList<StorageType>(Arrays.asList(storageTypes));
-      if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
-          existing, true)) {
-        List<StorageTypeNodePair> sourceWithStorageMap =
-            new ArrayList<StorageTypeNodePair>();
-        List<DatanodeStorageInfo> existingBlockStorages =
-            new ArrayList<DatanodeStorageInfo>(Arrays.asList(storages));
-        for (StorageType existingType : existing) {
-          Iterator<DatanodeStorageInfo> iterator =
-              existingBlockStorages.iterator();
-          while (iterator.hasNext()) {
-            DatanodeStorageInfo datanodeStorageInfo = iterator.next();
-            StorageType storageType = datanodeStorageInfo.getStorageType();
-            if (storageType == existingType) {
-              iterator.remove();
-              sourceWithStorageMap.add(new StorageTypeNodePair(storageType,
-                  datanodeStorageInfo.getDatanodeDescriptor()));
-              break;
-            }
-          }
-        }
+      List<StorageType> expectedStorageTypes = existingStoragePolicy
+            .chooseStorageTypes(blockInfo.getReplication());
+      foundMatchingTargetNodesForAllBlocks |= computeBlockMovingInfos(
+          blockMovingInfos, blockInfo, expectedStorageTypes);
+    }
 
-        StorageTypeNodeMap locsForExpectedStorageTypes =
-            findTargetsForExpectedStorageTypes(expectedStorageTypes);
-
-        BlockMovingInfo blockMovingInfo =
-            findSourceAndTargetToMove(blockInfo, existing, sourceWithStorageMap,
-                expectedStorageTypes, locsForExpectedStorageTypes);
-        if (coordinatorNode == null) {
-          // For now, first datanode will be chosen as the co-ordinator. Later
-          // this can be optimized if needed.
-          coordinatorNode =
-              (DatanodeDescriptor) blockMovingInfo.getSources()[0];
+    assignBlockMovingInfosToCoordinatorDn(blockCollection.getId(),
+        blockMovingInfos, coordinatorNode);
+    return foundMatchingTargetNodesForAllBlocks;
+  }
+
+  /**
+   * Compute the list of block moving information corresponding to the given
+   * blockId. This will check that each block location of the given block is
+   * satisfying the expected storage policy. If block location is not satisfied
+   * the policy then find out the target node with the expected storage type to
+   * satisfy the storage policy.
+   *
+   * @param blockMovingInfos
+   *          - list of block source and target node pair
+   * @param blockInfo
+   *          - block details
+   * @param expectedStorageTypes
+   *          - list of expected storage type to satisfy the storage policy
+   * @return false if some of the block locations failed to find target node to
+   *         satisfy the storage policy, true otherwise
+   */
+  private boolean computeBlockMovingInfos(
+      List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
+      List<StorageType> expectedStorageTypes) {
+    boolean foundMatchingTargetNodesForBlock = true;
+    DatanodeStorageInfo[] storages = blockManager.getStorages(blockInfo);
+    StorageType[] storageTypes = new StorageType[storages.length];
+    for (int j = 0; j < storages.length; j++) {
+      DatanodeStorageInfo datanodeStorageInfo = storages[j];
+      StorageType storageType = datanodeStorageInfo.getStorageType();
+      storageTypes[j] = storageType;
+    }
+    List<StorageType> existing =
+        new LinkedList<StorageType>(Arrays.asList(storageTypes));
+    if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
+        existing, true)) {
+      List<StorageTypeNodePair> sourceWithStorageMap =
+          new ArrayList<StorageTypeNodePair>();
+      List<DatanodeStorageInfo> existingBlockStorages =
+          new ArrayList<DatanodeStorageInfo>(Arrays.asList(storages));
+      for (StorageType existingType : existing) {
+        Iterator<DatanodeStorageInfo> iterator =
+            existingBlockStorages.iterator();
+        while (iterator.hasNext()) {
+          DatanodeStorageInfo datanodeStorageInfo = iterator.next();
+          StorageType storageType = datanodeStorageInfo.getStorageType();
+          if (storageType == existingType) {
+            iterator.remove();
+            sourceWithStorageMap.add(new StorageTypeNodePair(storageType,
+                datanodeStorageInfo.getDatanodeDescriptor()));
+            break;
+          }
         }
-        blockMovingInfos.add(blockMovingInfo);
       }
-    }
 
-    addBlockMovingInfosToCoordinatorDn(blockCollectionID, blockMovingInfos,
-        coordinatorNode);
+      StorageTypeNodeMap locsForExpectedStorageTypes =
+          findTargetsForExpectedStorageTypes(expectedStorageTypes);
+
+      foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
+          blockMovingInfos, blockInfo, existing, sourceWithStorageMap,
+          expectedStorageTypes, locsForExpectedStorageTypes);
+    }
+    return foundMatchingTargetNodesForBlock;
   }
 
-  private void addBlockMovingInfosToCoordinatorDn(long blockCollectionID,
+  private void assignBlockMovingInfosToCoordinatorDn(long blockCollectionID,
       List<BlockMovingInfo> blockMovingInfos,
       DatanodeDescriptor coordinatorNode) {
 
@@ -278,6 +308,11 @@ public class StoragePolicySatisfier implements Runnable {
       return;
     }
 
+    // For now, first datanode will be chosen as the co-ordinator. Later
+    // this can be optimized if needed.
+    coordinatorNode = (DatanodeDescriptor) blockMovingInfos.get(0)
+        .getSources()[0];
+
     boolean needBlockStorageMovement = false;
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       // Check for atleast one block storage movement has been chosen
@@ -301,6 +336,8 @@ public class StoragePolicySatisfier implements Runnable {
    * Find the good target node for each source node for which block storages was
    * misplaced.
    *
+   * @param blockMovingInfos
+   *          - list of block source and target node pair
    * @param blockInfo
    *          - Block
    * @param existing
@@ -311,23 +348,49 @@ public class StoragePolicySatisfier implements Runnable {
    *          - Expecting storages to move
    * @param locsForExpectedStorageTypes
    *          - Available DNs for expected storage types
-   * @return list of block source and target node pair
+   * @return false if some of the block locations failed to find target node to
+   *         satisfy the storage policy
    */
-  private BlockMovingInfo findSourceAndTargetToMove(BlockInfo blockInfo,
+  private boolean findSourceAndTargetToMove(
+      List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
       List<StorageType> existing,
       List<StorageTypeNodePair> sourceWithStorageList,
       List<StorageType> expected,
       StorageTypeNodeMap locsForExpectedStorageTypes) {
+    boolean foundMatchingTargetNodesForBlock = true;
     List<DatanodeInfo> sourceNodes = new ArrayList<>();
     List<StorageType> sourceStorageTypes = new ArrayList<>();
     List<DatanodeInfo> targetNodes = new ArrayList<>();
     List<StorageType> targetStorageTypes = new ArrayList<>();
     List<DatanodeDescriptor> chosenNodes = new ArrayList<>();
+
+    // Looping over all the source node locations and choose the target
+    // storage within same node if possible. This is done separately to
+    // avoid choosing a target which already has this block.
     for (int i = 0; i < sourceWithStorageList.size(); i++) {
       StorageTypeNodePair existingTypeNodePair = sourceWithStorageList.get(i);
       StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(
           existingTypeNodePair.dn, expected);
+      if (chosenTarget != null) {
+        sourceNodes.add(existingTypeNodePair.dn);
+        sourceStorageTypes.add(existingTypeNodePair.storageType);
+        targetNodes.add(chosenTarget.dn);
+        targetStorageTypes.add(chosenTarget.storageType);
+        chosenNodes.add(chosenTarget.dn);
+        // TODO: We can increment scheduled block count for this node?
+      }
+    }
 
+    // Looping over all the source node locations. Choose a remote target
+    // storage node if it was not found out within same node.
+    for (int i = 0; i < sourceWithStorageList.size(); i++) {
+      StorageTypeNodePair existingTypeNodePair = sourceWithStorageList.get(i);
+      StorageTypeNodePair chosenTarget = null;
+      // Chosen the target storage within same datanode. So just skipping this
+      // source node.
+      if (sourceNodes.contains(existingTypeNodePair.dn)) {
+        continue;
+      }
       if (chosenTarget == null && blockManager.getDatanodeManager()
           .getNetworkTopology().isNodeGroupAware()) {
         chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
@@ -359,18 +422,40 @@ public class StoragePolicySatisfier implements Runnable {
             "Failed to choose target datanode for the required"
                 + " storage types {}, block:{}, existing storage type:{}",
             expected, blockInfo, existingTypeNodePair.storageType);
-        sourceNodes.add(existingTypeNodePair.dn);
-        sourceStorageTypes.add(existingTypeNodePair.storageType);
-        // Imp: Not setting the target details, empty targets. Later, this is
-        // used as an indicator for retrying this block movement.
+        foundMatchingTargetNodesForBlock = false;
       }
     }
-    BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blockInfo,
+
+    blockMovingInfos.addAll(getBlockMovingInfos(blockInfo, sourceNodes,
+        sourceStorageTypes, targetNodes, targetStorageTypes));
+    return foundMatchingTargetNodesForBlock;
+  }
+
+  private List<BlockMovingInfo> getBlockMovingInfos(BlockInfo blockInfo,
+      List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
+      List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes) {
+    List<BlockMovingInfo> blkMovingInfos = new ArrayList<>();
+    // No source-target node pair exists.
+    if (sourceNodes.size() <= 0) {
+      return blkMovingInfos;
+    }
+    buildBlockMovingInfos(blockInfo, sourceNodes, sourceStorageTypes,
+        targetNodes, targetStorageTypes, blkMovingInfos);
+    return blkMovingInfos;
+  }
+
+  private void buildBlockMovingInfos(BlockInfo blockInfo,
+      List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
+      List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes,
+      List<BlockMovingInfo> blkMovingInfos) {
+    Block blk = new Block(blockInfo.getBlockId(), blockInfo.getNumBytes(),
+        blockInfo.getGenerationStamp());
+    BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk,
         sourceNodes.toArray(new DatanodeInfo[sourceNodes.size()]),
         targetNodes.toArray(new DatanodeInfo[targetNodes.size()]),
         sourceStorageTypes.toArray(new StorageType[sourceStorageTypes.size()]),
         targetStorageTypes.toArray(new StorageType[targetStorageTypes.size()]));
-    return blkMovingInfo;
+    blkMovingInfos.add(blkMovingInfo);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe2cb59f/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 8c70d99..6641134 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
@@ -33,13 +33,13 @@ public class TestBlockStorageMovementAttemptedItems {
 
   private BlockStorageMovementAttemptedItems bsmAttemptedItems = null;
   private BlockStorageMovementNeeded unsatisfiedStorageMovementFiles = null;
+  private final int selfRetryTimeout = 500;
 
   @Before
   public void setup() {
     unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded();
-    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100, 500,
-        unsatisfiedStorageMovementFiles);
-    bsmAttemptedItems.start();
+    bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
+        selfRetryTimeout, unsatisfiedStorageMovementFiles);
   }
 
   @After
@@ -72,8 +72,9 @@ public class TestBlockStorageMovementAttemptedItems {
 
   @Test(timeout = 30000)
   public void testAddResultWithFailureResult() throws Exception {
+    bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item);
+    bsmAttemptedItems.add(item, true);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
             item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
@@ -82,8 +83,9 @@ public class TestBlockStorageMovementAttemptedItems {
 
   @Test(timeout = 30000)
   public void testAddResultWithSucessResult() throws Exception {
+    bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item);
+    bsmAttemptedItems.add(item, true);
     bsmAttemptedItems.addResults(
         new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
             item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
@@ -92,10 +94,93 @@ public class TestBlockStorageMovementAttemptedItems {
 
   @Test(timeout = 30000)
   public void testNoResultAdded() throws Exception {
+    bsmAttemptedItems.start(); // start block movement result monitor thread
     Long item = new Long(1234);
-    bsmAttemptedItems.add(item);
-    // After selfretry timeout, it should be added back for retry
-    assertTrue(checkItemMovedForRetry(item, 600));
+    bsmAttemptedItems.add(item, true);
+    // After self retry timeout, it should be added back for retry
+    assertTrue("Failed to add to the retry list",
+        checkItemMovedForRetry(item, 600));
+    assertEquals("Failed to remove from the attempted list", 0,
+        bsmAttemptedItems.getAttemptedItemsCount());
   }
 
+  /**
+   * Partial block movement with BlocksStorageMovementResult#SUCCESS. Here,
+   * first occurrence is #blockStorageMovementResultCheck() and then
+   * #blocksStorageMovementUnReportedItemsCheck().
+   */
+  @Test(timeout = 30000)
+  public void testPartialBlockMovementShouldBeRetried1() throws Exception {
+    Long item = new Long(1234);
+    bsmAttemptedItems.add(item, false);
+    bsmAttemptedItems.addResults(
+        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
+            item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
+
+    // start block movement result monitor thread
+    bsmAttemptedItems.start();
+    assertTrue("Failed to add to the retry list",
+        checkItemMovedForRetry(item, 5000));
+    assertEquals("Failed to remove from the attempted list", 0,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
+
+  /**
+   * Partial block movement with BlocksStorageMovementResult#SUCCESS. Here,
+   * first occurrence is #blocksStorageMovementUnReportedItemsCheck() and then
+   * #blockStorageMovementResultCheck().
+   */
+  @Test(timeout = 30000)
+  public void testPartialBlockMovementShouldBeRetried2() throws Exception {
+    Long item = new Long(1234);
+    bsmAttemptedItems.add(item, false);
+    bsmAttemptedItems.addResults(
+        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
+            item.longValue(), BlocksStorageMovementResult.Status.SUCCESS)});
+
+    Thread.sleep(selfRetryTimeout * 2); // Waiting to get timed out
+
+    bsmAttemptedItems.blocksStorageMovementUnReportedItemsCheck();
+    bsmAttemptedItems.blockStorageMovementResultCheck();
+
+    assertTrue("Failed to add to the retry list",
+        checkItemMovedForRetry(item, 5000));
+    assertEquals("Failed to remove from the attempted list", 0,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
+
+  /**
+   * Partial block movement with only BlocksStorageMovementResult#FAILURE result
+   * and storageMovementAttemptedItems list is empty.
+   */
+  @Test(timeout = 30000)
+  public void testPartialBlockMovementShouldBeRetried3() throws Exception {
+    Long item = new Long(1234);
+    bsmAttemptedItems.addResults(
+        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
+            item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
+    bsmAttemptedItems.blockStorageMovementResultCheck();
+    assertTrue("Failed to add to the retry list",
+        checkItemMovedForRetry(item, 5000));
+    assertEquals("Failed to remove from the attempted list", 0,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
+
+  /**
+   * Partial block movement with BlocksStorageMovementResult#FAILURE result and
+   * storageMovementAttemptedItems.
+   */
+  @Test(timeout = 30000)
+  public void testPartialBlockMovementShouldBeRetried4() throws Exception {
+    Long item = new Long(1234);
+    bsmAttemptedItems.add(item, false);
+    bsmAttemptedItems.addResults(
+        new BlocksStorageMovementResult[]{new BlocksStorageMovementResult(
+            item.longValue(), BlocksStorageMovementResult.Status.FAILURE)});
+    bsmAttemptedItems.blockStorageMovementResultCheck();
+    assertTrue("Failed to add to the retry list",
+        checkItemMovedForRetry(item, 5000));
+    assertEquals("Failed to remove from the attempted list", 0,
+        bsmAttemptedItems.getAttemptedItemsCount());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe2cb59f/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 179b66b..718dbcb 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
@@ -527,6 +527,59 @@ public class TestStoragePolicySatisfier {
     waitExpectedStorageType(file1, StorageType.DISK, 2, 30000);
   }
 
+  /**
+   * Tests to verify that for the given path, only few of the blocks or block
+   * src locations(src nodes) under the given path will be scheduled for block
+   * movement.
+   *
+   * For example, there are two block for a file:
+   *
+   * File1 => two blocks and default storage policy(HOT).
+   * blk_1[locations=A(DISK),B(DISK),C(DISK),D(DISK),E(DISK)],
+   * blk_2[locations=A(DISK),B(DISK),C(DISK),D(DISK),E(DISK)].
+   *
+   * Now, set storage policy to COLD.
+   * Only two Dns are available with expected storage type ARCHIVE, say A, E.
+   *
+   * SPS will schedule block movement to the coordinator node with the details,
+   * blk_1[move A(DISK) -> A(ARCHIVE), move E(DISK) -> E(ARCHIVE)],
+   * blk_2[move A(DISK) -> A(ARCHIVE), move E(DISK) -> E(ARCHIVE)].
+   */
+  @Test(timeout = 300000)
+  public void testWhenOnlyFewSourceNodesHaveMatchingTargetNodes()
+      throws Exception {
+    try {
+      int numOfDns = 5;
+      config.setLong("dfs.block.size", 1024);
+      allDiskTypes =
+          new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+              {StorageType.DISK, StorageType.DISK},
+              {StorageType.DISK, StorageType.DISK},
+              {StorageType.DISK, StorageType.DISK},
+              {StorageType.DISK, StorageType.ARCHIVE}};
+      hdfsCluster = startCluster(config, allDiskTypes, numOfDns,
+          storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(file, (short) 5);
+
+      // Change policy to COLD
+      dfs.setStoragePolicy(new Path(file), "COLD");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier identified that block to move to
+      // ARCHIVE area.
+      waitExpectedStorageType(file, StorageType.ARCHIVE, 2, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, 3, 30000);
+
+      waitForBlocksMovementResult(1, 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
       throws IOException {
     ArrayList<DataNode> dns = hdfsCluster.getDataNodes();
@@ -561,7 +614,7 @@ public class TestStoragePolicySatisfier {
       DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
       favoredNodesCount--;
       if (favoredNodesCount <= 0) {
-        break;// marked favoredNodesCount number of pinned block location
+        break; // marked favoredNodesCount number of pinned block location
       }
     }
     return file1;
@@ -600,8 +653,14 @@ public class TestStoragePolicySatisfier {
   }
 
   private void writeContent(final String fileName) throws IOException {
+    writeContent(fileName, (short) 3);
+  }
+
+  private void writeContent(final String fileName, short replicatonFactor)
+      throws IOException {
     // write to DISK
-    final FSDataOutputStream out = dfs.create(new Path(fileName));
+    final FSDataOutputStream out = dfs.create(new Path(fileName),
+        replicatonFactor);
     for (int i = 0; i < 1000; i++) {
       out.writeChars("t");
     }


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


[06/50] [abbrv] hadoop git commit: HDFS-10884: [SPS]: Add block movement tracker to track the completion of block movement future tasks at DN. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-10884: [SPS]: Add block movement tracker to track the completion of block movement future tasks at DN. 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/7f9eb149
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7f9eb149
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7f9eb149

Branch: refs/heads/HDFS-10285
Commit: 7f9eb1491a569d97b3e6fa5ab33c102f94687db5
Parents: 4eaf873
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue Oct 25 00:40:45 2016 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:20:07 2017 +0530

----------------------------------------------------------------------
 .../datanode/BlockStorageMovementTracker.java   | 146 ++++++++++++++
 .../datanode/StoragePolicySatisfyWorker.java    | 178 +++++++++++++----
 .../protocol/BlockStorageMovementCommand.java   |  12 +-
 .../TestStoragePolicySatisfyWorker.java         | 190 +++++++++++++------
 4 files changed, 427 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f9eb149/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
new file mode 100644
index 0000000..d31f075
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlocksMovementsCompletionHandler;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used to track the completion of block movement future tasks.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class BlockStorageMovementTracker implements Runnable {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(BlockStorageMovementTracker.class);
+  private final CompletionService<BlockMovementResult> moverCompletionService;
+  private final BlocksMovementsCompletionHandler blksMovementscompletionHandler;
+
+  // Keeps the information - trackID vs its list of blocks
+  private final Map<Long, List<Future<BlockMovementResult>>> moverTaskFutures;
+  private final Map<Long, List<BlockMovementResult>> movementResults;
+
+  /**
+   * BlockStorageMovementTracker constructor.
+   *
+   * @param moverCompletionService
+   *          completion service.
+   * @param handler
+   *          blocks movements completion handler
+   */
+  public BlockStorageMovementTracker(
+      CompletionService<BlockMovementResult> moverCompletionService,
+      BlocksMovementsCompletionHandler handler) {
+    this.moverCompletionService = moverCompletionService;
+    this.moverTaskFutures = new HashMap<>();
+    this.blksMovementscompletionHandler = handler;
+    this.movementResults = new HashMap<>();
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (moverTaskFutures.size() <= 0) {
+        try {
+          synchronized (moverTaskFutures) {
+            // Waiting for mover tasks.
+            moverTaskFutures.wait(2000);
+          }
+        } catch (InterruptedException ignore) {
+          // ignore
+        }
+      }
+      try {
+        Future<BlockMovementResult> future = moverCompletionService.take();
+        if (future != null) {
+          BlockMovementResult result = future.get();
+          LOG.debug("Completed block movement. {}", result);
+          long trackId = result.getTrackId();
+          List<Future<BlockMovementResult>> blocksMoving = moverTaskFutures
+              .get(trackId);
+          blocksMoving.remove(future);
+
+          List<BlockMovementResult> resultPerTrackIdList =
+              addMovementResultToTrackIdList(result);
+
+          // Completed all the scheduled blocks movement under this 'trackId'.
+          if (blocksMoving.isEmpty()) {
+            synchronized (moverTaskFutures) {
+              moverTaskFutures.remove(trackId);
+            }
+            // handle completed blocks movements per trackId.
+            blksMovementscompletionHandler.handle(resultPerTrackIdList);
+            movementResults.remove(trackId);
+          }
+        }
+      } catch (ExecutionException | InterruptedException e) {
+        // TODO: Do we need failure retries and implement the same if required.
+        LOG.error("Exception while moving block replica to target storage type",
+            e);
+      }
+    }
+  }
+
+  private List<BlockMovementResult> addMovementResultToTrackIdList(
+      BlockMovementResult result) {
+    long trackId = result.getTrackId();
+    List<BlockMovementResult> perTrackIdList = movementResults.get(trackId);
+    if (perTrackIdList == null) {
+      perTrackIdList = new ArrayList<>();
+      movementResults.put(trackId, perTrackIdList);
+    }
+    perTrackIdList.add(result);
+    return perTrackIdList;
+  }
+
+  /**
+   * Add future task to the tracking list to check the completion status of the
+   * block movement.
+   *
+   * @param trackID
+   *          tracking Id
+   * @param futureTask
+   *          future task used for moving the respective block
+   */
+  void addBlock(long trackID, Future<BlockMovementResult> futureTask) {
+    synchronized (moverTaskFutures) {
+      List<Future<BlockMovementResult>> futures = moverTaskFutures
+          .get(Long.valueOf(trackID));
+      // null for the first task
+      if (futures == null) {
+        futures = new ArrayList<>();
+        moverTaskFutures.put(trackID, futures);
+      }
+      futures.add(futureTask);
+      // Notify waiting tracker thread about the newly added tasks.
+      moverTaskFutures.notify();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f9eb149/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 2c99963..604fb4a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -33,7 +33,6 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -65,6 +64,8 @@ import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * StoragePolicySatisfyWorker handles the storage policy satisfier commands.
  * These commands would be issued from NameNode as part of Datanode's heart beat
@@ -82,8 +83,10 @@ public class StoragePolicySatisfyWorker {
 
   private final int moverThreads;
   private final ExecutorService moveExecutor;
-  private final CompletionService<Void> moverExecutorCompletionService;
-  private final List<Future<Void>> moverTaskFutures;
+  private final CompletionService<BlockMovementResult> moverCompletionService;
+  private final BlocksMovementsCompletionHandler handler;
+  private final BlockStorageMovementTracker movementTracker;
+  private Daemon movementTrackerThread;
 
   public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
     this.datanode = datanode;
@@ -92,9 +95,13 @@ public class StoragePolicySatisfyWorker {
     moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
         DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
     moveExecutor = initializeBlockMoverThreadPool(moverThreads);
-    moverExecutorCompletionService = new ExecutorCompletionService<>(
-        moveExecutor);
-    moverTaskFutures = new ArrayList<>();
+    moverCompletionService = new ExecutorCompletionService<>(moveExecutor);
+    handler = new BlocksMovementsCompletionHandler();
+    movementTracker = new BlockStorageMovementTracker(moverCompletionService,
+        handler);
+    movementTrackerThread = new Daemon(movementTracker);
+    movementTrackerThread.setName("BlockStorageMovementTracker");
+    movementTrackerThread.start();
     // TODO: Needs to manage the number of concurrent moves per DataNode.
   }
 
@@ -133,10 +140,6 @@ public class StoragePolicySatisfyWorker {
    * separate thread. Each task will move the block replica to the target node &
    * wait for the completion.
    *
-   * TODO: Presently this function is a blocking call, this has to be refined by
-   * moving the tracking logic to another tracker thread. HDFS-10884 jira
-   * addresses the same.
-   *
    * @param trackID
    *          unique tracking identifier
    * @param blockPoolID
@@ -146,68 +149,64 @@ public class StoragePolicySatisfyWorker {
    */
   public void processBlockMovingTasks(long trackID, String blockPoolID,
       Collection<BlockMovingInfo> blockMovingInfos) {
-    Future<Void> moveCallable = null;
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       assert blkMovingInfo
           .getSources().length == blkMovingInfo.getTargets().length;
 
       for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
         BlockMovingTask blockMovingTask = new BlockMovingTask(
-            blkMovingInfo.getBlock(), blockPoolID,
+            trackID, blockPoolID, blkMovingInfo.getBlock(),
             blkMovingInfo.getSources()[i], blkMovingInfo.getTargets()[i],
+            blkMovingInfo.getSourceStorageTypes()[i],
             blkMovingInfo.getTargetStorageTypes()[i]);
-        moveCallable = moverExecutorCompletionService.submit(blockMovingTask);
-        moverTaskFutures.add(moveCallable);
-      }
-    }
-
-    for (int i = 0; i < moverTaskFutures.size(); i++) {
-      try {
-        moveCallable = moverExecutorCompletionService.take();
-        moveCallable.get();
-      } catch (InterruptedException | ExecutionException e) {
-        // TODO: Failure retries and report back the error to NameNode.
-        LOG.error("Exception while moving block replica to target storage type",
-            e);
+        Future<BlockMovementResult> moveCallable = moverCompletionService
+            .submit(blockMovingTask);
+        movementTracker.addBlock(trackID, moveCallable);
       }
     }
   }
 
   /**
    * This class encapsulates the process of moving the block replica to the
-   * given target.
+   * given target and wait for the response.
    */
-  private class BlockMovingTask implements Callable<Void> {
+  private class BlockMovingTask implements Callable<BlockMovementResult> {
+    private final long trackID;
+    private final String blockPoolID;
     private final Block block;
     private final DatanodeInfo source;
     private final DatanodeInfo target;
+    private final StorageType srcStorageType;
     private final StorageType targetStorageType;
-    private String blockPoolID;
 
-    BlockMovingTask(Block block, String blockPoolID, DatanodeInfo source,
-        DatanodeInfo target, StorageType targetStorageType) {
-      this.block = block;
+    BlockMovingTask(long trackID, String blockPoolID, Block block,
+        DatanodeInfo source, DatanodeInfo target,
+        StorageType srcStorageType, StorageType targetStorageType) {
+      this.trackID = trackID;
       this.blockPoolID = blockPoolID;
+      this.block = block;
       this.source = source;
       this.target = target;
+      this.srcStorageType = srcStorageType;
       this.targetStorageType = targetStorageType;
     }
 
     @Override
-    public Void call() {
-      moveBlock();
-      return null;
+    public BlockMovementResult call() {
+      BlockMovementStatus status = moveBlock();
+      return new BlockMovementResult(trackID, block.getBlockId(), target,
+          status);
     }
 
-    private void moveBlock() {
-      LOG.info("Start moving block {}", block);
-
-      LOG.debug("Start moving block:{} from src:{} to destin:{} to satisfy "
-          + "storageType:{}", block, source, target, targetStorageType);
+    private BlockMovementStatus moveBlock() {
+      LOG.info("Start moving block:{} from src:{} to destin:{} to satisfy "
+              + "storageType, sourceStoragetype:{} and destinStoragetype:{}",
+          block, source, target, srcStorageType, targetStorageType);
       Socket sock = null;
       DataOutputStream out = null;
       DataInputStream in = null;
       try {
+        ExtendedBlock extendedBlock = new ExtendedBlock(blockPoolID, block);
         DNConf dnConf = datanode.getDnConf();
         String dnAddr = target.getXferAddr(dnConf.getConnectToDnViaHostname());
         sock = datanode.newSocket();
@@ -218,7 +217,6 @@ public class StoragePolicySatisfyWorker {
 
         OutputStream unbufOut = sock.getOutputStream();
         InputStream unbufIn = sock.getInputStream();
-        ExtendedBlock extendedBlock = new ExtendedBlock(blockPoolID, block);
         Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
             extendedBlock, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 
@@ -239,12 +237,14 @@ public class StoragePolicySatisfyWorker {
             "Successfully moved block:{} from src:{} to destin:{} for"
                 + " satisfying storageType:{}",
             block, source, target, targetStorageType);
+        return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_SUCCESS;
       } catch (IOException e) {
         // TODO: handle failure retries
         LOG.warn(
             "Failed to move block:{} from src:{} to destin:{} to satisfy "
                 + "storageType:{}",
             block, source, target, targetStorageType, e);
+        return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE;
       } finally {
         IOUtils.closeStream(out);
         IOUtils.closeStream(in);
@@ -272,4 +272,102 @@ public class StoragePolicySatisfyWorker {
       DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
     }
   }
+
+  /**
+   * Block movement status code.
+   */
+  enum BlockMovementStatus {
+    /** Success. */
+    DN_BLK_STORAGE_MOVEMENT_SUCCESS(0),
+    /**
+     * Failure due to generation time stamp mismatches or network errors
+     * or no available space.
+     */
+    DN_BLK_STORAGE_MOVEMENT_FAILURE(-1);
+
+    // TODO: need to support different type of failures. Failure due to network
+    // errors, block pinned, no space available etc.
+
+    private final int code;
+
+    private BlockMovementStatus(int code) {
+      this.code = code;
+    }
+
+    /**
+     * @return the status code.
+     */
+    int getStatusCode() {
+      return code;
+    }
+  }
+
+  /**
+   * This class represents result from a block movement task. This will have the
+   * information of the task which was successful or failed due to errors.
+   */
+  static class BlockMovementResult {
+    private final long trackId;
+    private final long blockId;
+    private final DatanodeInfo target;
+    private final BlockMovementStatus status;
+
+    public BlockMovementResult(long trackId, long blockId,
+        DatanodeInfo target, BlockMovementStatus status) {
+      this.trackId = trackId;
+      this.blockId = blockId;
+      this.target = target;
+      this.status = status;
+    }
+
+    long getTrackId() {
+      return trackId;
+    }
+
+    long getBlockId() {
+      return blockId;
+    }
+
+    BlockMovementStatus getStatus() {
+      return status;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("Block movement result(\n  ")
+          .append("track id: ").append(trackId).append(" block id: ")
+          .append(blockId).append(" target node: ").append(target)
+          .append(" movement status: ").append(status).append(")").toString();
+    }
+  }
+
+  /**
+   * Blocks movements completion handler, which is used to collect details of
+   * the completed list of block movements and notify the namenode about the
+   * success or failures.
+   */
+  static class BlocksMovementsCompletionHandler {
+    private final List<BlockMovementResult> completedBlocks = new ArrayList<>();
+
+    /**
+     * Collect all the block movement results and notify namenode.
+     *
+     * @param results
+     *          result of all the block movements per trackId
+     */
+    void handle(List<BlockMovementResult> results) {
+      completedBlocks.addAll(results);
+      // TODO: notify namenode about the success/failures.
+    }
+
+    @VisibleForTesting
+    List<BlockMovementResult> getCompletedBlocks() {
+      return completedBlocks;
+    }
+  }
+
+  @VisibleForTesting
+  BlocksMovementsCompletionHandler getBlocksMovementsCompletionHandler() {
+    return handler;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f9eb149/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
index 7c97f1a..5dcf4e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -35,10 +35,10 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
  * service. After the block movement this DataNode sends response back to the
  * NameNode about the movement status.
  *
- * The coordinator datanode will use 'trackId' identifier to coordinate the block
- * movement of the given set of blocks. TrackId is a unique identifier that
- * represents a group of blocks. Namenode will generate this unique value and
- * send it to the coordinator datanode along with the
+ * The coordinator datanode will use 'trackId' identifier to coordinate the
+ * block movement of the given set of blocks. TrackId is a unique identifier
+ * that represents a group of blocks. Namenode will generate this unique value
+ * and send it to the coordinator datanode along with the
  * BlockStorageMovementCommand. Datanode will monitor the completion of the
  * block movements that grouped under this trackId and notifies Namenode about
  * the completion status.
@@ -153,11 +153,11 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
       return new StringBuilder().append("BlockMovingInfo(\n  ")
           .append("Moving block: ").append(blk).append(" From: ")
           .append(Arrays.asList(sourceNodes)).append(" To: [")
-          .append(Arrays.asList(targetNodes)).append(")\n")
+          .append(Arrays.asList(targetNodes)).append("\n  ")
           .append(" sourceStorageTypes: ")
           .append(Arrays.toString(sourceStorageTypes))
           .append(" targetStorageTypes: ")
-          .append(Arrays.toString(targetStorageTypes)).toString();
+          .append(Arrays.toString(targetStorageTypes)).append(")").toString();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f9eb149/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 d803f1a..ea3eec3 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -33,10 +34,15 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementResult;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,8 +57,9 @@ public class TestStoragePolicySatisfyWorker {
 
   private static final Logger LOG = LoggerFactory
       .getLogger(TestStoragePolicySatisfyWorker.class);
-
   private static final int DEFAULT_BLOCK_SIZE = 100;
+  private MiniDFSCluster cluster = null;
+  private final Configuration conf = new HdfsConfiguration();
 
   private static void initConf(Configuration conf) {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
@@ -63,64 +70,141 @@ public class TestStoragePolicySatisfyWorker {
     conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
   }
 
+  @Before
+  public void setUp() throws IOException {
+    initConf(conf);
+  }
+
+  @After
+  public void teardown() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
   /**
    * Tests to verify that the block replica is moving to ARCHIVE storage type to
    * fulfill the storage policy requirement.
    */
   @Test(timeout = 120000)
   public void testMoveSingleBlockToAnotherDatanode() throws Exception {
-    final Configuration conf = new HdfsConfiguration();
-    initConf(conf);
-    final MiniDFSCluster cluster =
-        new MiniDFSCluster.Builder(conf).numDataNodes(4)
-            .storageTypes(
-                new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
-                    {StorageType.DISK, StorageType.ARCHIVE},
-                    {StorageType.ARCHIVE, StorageType.ARCHIVE},
-                    {StorageType.ARCHIVE, StorageType.ARCHIVE}})
-            .build();
-    try {
-      cluster.waitActive();
-      final DistributedFileSystem dfs = cluster.getFileSystem();
-      final String file = "/testMoveSingleBlockToAnotherDatanode";
-      // write to DISK
-      final FSDataOutputStream out = dfs.create(new Path(file), (short) 2);
-      out.writeChars("testMoveSingleBlockToAnotherDatanode");
-      out.close();
-
-      // verify before movement
-      LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
-      StorageType[] storageTypes = lb.getStorageTypes();
-      for (StorageType storageType : storageTypes) {
-        Assert.assertTrue(StorageType.DISK == storageType);
-      }
-      // move to ARCHIVE
-      dfs.setStoragePolicy(new Path(file), "COLD");
-
-      lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
-      DataNode src = cluster.getDataNodes().get(3);
-      DatanodeInfo targetDnInfo = DFSTestUtil
-          .getLocalDatanodeInfo(src.getXferPort());
-
-      // TODO: Need to revisit this when NN is implemented to be able to send
-      // block moving commands.
-      StoragePolicySatisfyWorker worker = new StoragePolicySatisfyWorker(conf,
-          src);
-      List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
-      BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
-          lb.getBlock().getLocalBlock(), lb.getLocations()[0], targetDnInfo,
-          lb.getStorageTypes()[0], StorageType.ARCHIVE);
-      blockMovingInfos.add(blockMovingInfo);
-      INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
-      worker.processBlockMovingTasks(inode.getId(),
-          cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
-      cluster.triggerHeartbeats();
-
-      // Wait till NameNode notified about the block location details
-      waitForLocatedBlockWithArchiveStorageType(dfs, file, 1, 30000);
-    } finally {
-      cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.ARCHIVE, StorageType.ARCHIVE},
+                {StorageType.ARCHIVE, StorageType.ARCHIVE}})
+        .build();
+    cluster.waitActive();
+    final DistributedFileSystem dfs = cluster.getFileSystem();
+    final String file = "/testMoveSingleBlockToAnotherDatanode";
+    // write to DISK
+    final FSDataOutputStream out = dfs.create(new Path(file), (short) 2);
+    out.writeChars("testMoveSingleBlockToAnotherDatanode");
+    out.close();
+
+    // verify before movement
+    LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+    StorageType[] storageTypes = lb.getStorageTypes();
+    for (StorageType storageType : storageTypes) {
+      Assert.assertTrue(StorageType.DISK == storageType);
+    }
+    // move to ARCHIVE
+    dfs.setStoragePolicy(new Path(file), "COLD");
+
+    FSNamesystem namesystem = cluster.getNamesystem();
+    INode inode = namesystem.getFSDirectory().getINode(file);
+    namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+
+    cluster.triggerHeartbeats();
+
+    // Wait till NameNode notified about the block location details
+    waitForLocatedBlockWithArchiveStorageType(dfs, file, 2, 30000);
+  }
+
+  /**
+   * Test to verify that satisfy worker can't move blocks. If specified target
+   * datanode doesn't have enough space to accommodate the moving block.
+   */
+  @Test(timeout = 120000)
+  public void testMoveWithNoSpaceAvailable() throws Exception {
+    final long capacity = 150;
+    final String rack0 = "/rack0";
+    final String rack1 = "/rack1";
+    long[] capacities = new long[] {capacity, capacity, capacity / 2};
+    String[] hosts = {"host0", "host1", "host2"};
+    String[] racks = {rack0, rack1, rack0};
+    int numOfDatanodes = capacities.length;
+
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numOfDatanodes)
+        .hosts(hosts).racks(racks).simulatedCapacities(capacities)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.ARCHIVE, StorageType.ARCHIVE}})
+        .build();
+
+    cluster.waitActive();
+    InetSocketAddress[] favoredNodes = new InetSocketAddress[3];
+    for (int i = 0; i < favoredNodes.length; i++) {
+      // DFSClient will attempt reverse lookup. In case it resolves
+      // "127.0.0.1" to "localhost", we manually specify the hostname.
+      favoredNodes[i] = cluster.getDataNodes().get(i).getXferAddress();
+    }
+    final DistributedFileSystem dfs = cluster.getFileSystem();
+    final String file = "/testMoveWithNoSpaceAvailable";
+    DFSTestUtil.createFile(dfs, new Path(file), false, 1024, 100,
+        DEFAULT_BLOCK_SIZE, (short) 2, 0, false, favoredNodes);
+
+    // verify before movement
+    LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+    StorageType[] storageTypes = lb.getStorageTypes();
+    for (StorageType storageType : storageTypes) {
+      Assert.assertTrue(StorageType.DISK == storageType);
     }
+
+    // move to ARCHIVE
+    dfs.setStoragePolicy(new Path(file), "COLD");
+
+    lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+    DataNode src = cluster.getDataNodes().get(2);
+    DatanodeInfo targetDnInfo = DFSTestUtil
+        .getLocalDatanodeInfo(src.getXferPort());
+
+    StoragePolicySatisfyWorker worker = new StoragePolicySatisfyWorker(conf,
+        src);
+    List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+    BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
+        lb.getBlock().getLocalBlock(), lb.getLocations()[0], targetDnInfo,
+        lb.getStorageTypes()[0], StorageType.ARCHIVE);
+    blockMovingInfos.add(blockMovingInfo);
+    INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
+    worker.processBlockMovingTasks(inode.getId(),
+        cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
+
+    waitForBlockMovementCompletion(worker, inode.getId(), 1, 30000);
+  }
+
+  private void waitForBlockMovementCompletion(
+      final StoragePolicySatisfyWorker worker, final long inodeId,
+      int expectedFailedItemsCount, int timeout) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        List<BlockMovementResult> completedBlocks = worker
+            .getBlocksMovementsCompletionHandler().getCompletedBlocks();
+        int failedCount = 0;
+        for (BlockMovementResult blockMovementResult : completedBlocks) {
+          if (BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE ==
+              blockMovementResult.getStatus()) {
+            failedCount++;
+          }
+        }
+        LOG.info("Block movement completed count={}, expected={} and actual={}",
+            completedBlocks.size(), expectedFailedItemsCount, failedCount);
+        return expectedFailedItemsCount == failedCount;
+      }
+    }, 100, timeout);
   }
 
   private void waitForLocatedBlockWithArchiveStorageType(
@@ -150,7 +234,7 @@ public class TestStoragePolicySatisfyWorker {
     }, 100, timeout);
   }
 
-  BlockMovingInfo prepareBlockMovingInfo(Block block,
+  private BlockMovingInfo prepareBlockMovingInfo(Block block,
       DatanodeInfo src, DatanodeInfo destin, StorageType storageType,
       StorageType targetStorageType) {
     return new BlockMovingInfo(block, new DatanodeInfo[] {src},


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


[04/50] [abbrv] hadoop git commit: HDFS-10800: [SPS]: Daemon thread in Namenode to find blocks placed in other storage than what the policy specifies. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-10800: [SPS]: Daemon thread in Namenode to find blocks placed in other storage than what the policy specifies. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: d0b52abb14ec9a69fa5bd420535d44eca514dc76
Parents: 6c118ea
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Fri Sep 23 13:41:29 2016 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:19:56 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  41 ++
 .../server/blockmanagement/BlockManager.java    |  20 +
 .../blockmanagement/DatanodeDescriptor.java     |  38 ++
 .../server/blockmanagement/DatanodeManager.java |   1 +
 .../datanode/StoragePolicySatisfyWorker.java    |  29 +-
 .../namenode/BlockStorageMovementNeeded.java    |  53 +++
 .../server/namenode/StoragePolicySatisfier.java | 397 +++++++++++++++++++
 .../protocol/BlockStorageMovementCommand.java   |  11 +-
 .../TestStoragePolicySatisfyWorker.java         |  24 +-
 .../namenode/TestStoragePolicySatisfier.java    | 209 ++++++++++
 10 files changed, 791 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 4c94e38..7465853 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -53,6 +53,7 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -73,6 +74,7 @@ import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -1681,4 +1683,43 @@ public class DFSUtil {
     }
     return id;
   }
+
+  /**
+   * Remove the overlap between the expected types and the existing types.
+   *
+   * @param expected
+   *          - Expected storage types list.
+   * @param existing
+   *          - Existing storage types list.
+   * @param ignoreNonMovable
+   *          ignore non-movable storage types by removing them from both
+   *          expected and existing storage type list to prevent non-movable
+   *          storage from being moved.
+   * @returns if the existing types or the expected types is empty after
+   *          removing the overlap.
+   */
+  public static boolean removeOverlapBetweenStorageTypes(
+      List<StorageType> expected,
+      List<StorageType> existing, boolean ignoreNonMovable) {
+    for (Iterator<StorageType> i = existing.iterator(); i.hasNext();) {
+      final StorageType t = i.next();
+      if (expected.remove(t)) {
+        i.remove();
+      }
+    }
+    if (ignoreNonMovable) {
+      removeNonMovable(existing);
+      removeNonMovable(expected);
+    }
+    return expected.isEmpty() || existing.isEmpty();
+  }
+
+  private static void removeNonMovable(List<StorageType> types) {
+    for (Iterator<StorageType> i = types.iterator(); i.hasNext();) {
+      final StorageType t = i.next();
+      if (!t.isMovable()) {
+        i.remove();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/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 ae04aac..3ae7b58 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
@@ -89,6 +89,8 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.namenode.BlockStorageMovementNeeded;
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
@@ -430,6 +432,11 @@ public class BlockManager implements BlockStatsMXBean {
 
   private final BlockIdManager blockIdManager;
 
+  /** For satisfying block storage policies */
+  private final StoragePolicySatisfier sps;
+  private final BlockStorageMovementNeeded storageMovementNeeded =
+      new BlockStorageMovementNeeded();
+
   /** Minimum live replicas needed for the datanode to be transitioned
    * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
    */
@@ -465,6 +472,7 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
         * 1000L);
 
+    sps = new StoragePolicySatisfier(namesystem, storageMovementNeeded, this);
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     this.maxCorruptFilesReturned = conf.getInt(
@@ -690,9 +698,11 @@ public class BlockManager implements BlockStatsMXBean {
     this.blockReportThread.start();
     mxBeanName = MBeans.register("NameNode", "BlockStats", this);
     bmSafeMode.activate(blockTotal);
+    sps.start();
   }
 
   public void close() {
+    sps.stop();
     bmSafeMode.close();
     try {
       redundancyThread.interrupt();
@@ -4902,4 +4912,14 @@ public class BlockManager implements BlockStatsMXBean {
   public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
     pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
   }
+
+  /**
+   * Set file block collection for which storage movement needed for its blocks.
+   *
+   * @param id
+   *          - file block collection id.
+   */
+  public void satisfyStoragePolicy(long id) {
+    storageMovementNeeded.add(id);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index d35894c..46a45ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -43,6 +43,8 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -205,6 +207,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final LightWeightHashSet<Block> invalidateBlocks =
       new LightWeightHashSet<>();
 
+  /** A queue of blocks for moving its storage placements by this datanode. */
+  private final Queue<List<BlockMovingInfo>> storageMovementBlocks =
+      new LinkedList<>();
+
   /* Variables for maintaining number of blocks scheduled to be written to
    * this storage. This count is approximate and might be slightly bigger
    * in case of errors (e.g. datanode does not report if an error occurs
@@ -1015,5 +1021,37 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
     return false;
   }
+
+  /**
+   * Add the block infos which needs to move its storage locations.
+   *
+   * @param storageMismatchedBlocks
+   *          - storage mismatched block infos
+   */
+  public void addBlocksToMoveStorage(
+      List<BlockMovingInfo> storageMismatchedBlocks) {
+    storageMovementBlocks.offer(storageMismatchedBlocks);
+  }
+
+  /**
+   * @return block infos which needs to move its storage locations.
+   */
+  public List<BlockMovingInfo> getBlocksToMoveStorages() {
+    return storageMovementBlocks.poll();
+  }
+
+  // TODO: we will remove this method once DN side handling integrated. We can
+  // convert the test to check real block movements instead of this ds.
+  @VisibleForTesting
+  public List<BlockMovingInfo> getStorageMovementPendingItems() {
+    List<BlockMovingInfo> flatList = new ArrayList<>();
+    Iterator<List<BlockMovingInfo>> iterator = storageMovementBlocks
+        .iterator();
+    while (iterator.hasNext()) {
+      List<BlockMovingInfo> next = iterator.next();
+      flatList.addAll(next);
+    }
+    return flatList;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index c75bcea..13c730d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 6df4e81..fa408f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
@@ -125,7 +126,7 @@ public class StoragePolicySatisfyWorker {
     return moverThreadPool;
   }
 
-  public void processBlockMovingTasks(long trackID,
+  public void processBlockMovingTasks(long trackID, String blockPoolID,
       List<BlockMovingInfo> blockMovingInfos) {
     Future<Void> moveCallable = null;
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
@@ -133,13 +134,11 @@ public class StoragePolicySatisfyWorker {
           .getSources().length == blkMovingInfo.getTargets().length;
 
       for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
-        BlockMovingTask blockMovingTask =
-            new BlockMovingTask(blkMovingInfo.getBlock(),
-            blkMovingInfo.getSources()[i],
-            blkMovingInfo.getTargets()[i],
+        BlockMovingTask blockMovingTask = new BlockMovingTask(
+            blkMovingInfo.getBlock(), blockPoolID,
+            blkMovingInfo.getSources()[i], blkMovingInfo.getTargets()[i],
             blkMovingInfo.getTargetStorageTypes()[i]);
-        moveCallable = moverExecutorCompletionService
-            .submit(blockMovingTask);
+        moveCallable = moverExecutorCompletionService.submit(blockMovingTask);
         moverTaskFutures.add(moveCallable);
       }
     }
@@ -163,14 +162,16 @@ public class StoragePolicySatisfyWorker {
    * given target.
    */
   private class BlockMovingTask implements Callable<Void> {
-    private final ExtendedBlock block;
+    private final Block block;
     private final DatanodeInfo source;
     private final DatanodeInfo target;
     private final StorageType targetStorageType;
+    private String blockPoolID;
 
-    BlockMovingTask(ExtendedBlock block, DatanodeInfo source,
+    BlockMovingTask(Block block, String blockPoolID, DatanodeInfo source,
         DatanodeInfo target, StorageType targetStorageType) {
       this.block = block;
+      this.blockPoolID = blockPoolID;
       this.source = source;
       this.target = target;
       this.targetStorageType = targetStorageType;
@@ -201,12 +202,12 @@ public class StoragePolicySatisfyWorker {
 
         OutputStream unbufOut = sock.getOutputStream();
         InputStream unbufIn = sock.getInputStream();
-
+        ExtendedBlock extendedBlock = new ExtendedBlock(blockPoolID, block);
         Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
-            block, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
+            extendedBlock, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 
         DataEncryptionKeyFactory keyFactory = datanode
-            .getDataEncryptionKeyFactoryForBlock(block);
+            .getDataEncryptionKeyFactoryForBlock(extendedBlock);
         IOStreamPair saslStreams = datanode.getSaslClient().socketSend(sock,
             unbufOut, unbufIn, keyFactory, accessToken, target);
         unbufOut = saslStreams.out;
@@ -215,10 +216,10 @@ public class StoragePolicySatisfyWorker {
             new BufferedOutputStream(unbufOut, ioFileBufferSize));
         in = new DataInputStream(
             new BufferedInputStream(unbufIn, ioFileBufferSize));
-        sendRequest(out, block, accessToken, source, targetStorageType);
+        sendRequest(out, extendedBlock, accessToken, source, targetStorageType);
         receiveResponse(in);
 
-        LOG.debug(
+        LOG.info(
             "Successfully moved block:{} from src:{} to destin:{} for"
                 + " satisfying storageType:{}",
             block, source, target, targetStorageType);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
new file mode 100644
index 0000000..c916672
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A Class to track the block collection IDs for which physical storage movement
+ * needed as per the Namespace and StorageReports from DN.
+ */
+@InterfaceAudience.Private
+public class BlockStorageMovementNeeded {
+  private final Queue<Long> storageMovementNeeded = new LinkedList<Long>();
+
+  /**
+   * Add the block collection id to tracking list for which storage movement
+   * expected if necessary.
+   *
+   * @param blockCollectionID
+   *          - block collection id, which is nothing but inode id.
+   */
+  public synchronized void add(Long blockCollectionID) {
+    storageMovementNeeded.add(blockCollectionID);
+  }
+
+  /**
+   * Gets the block collection id for which storage movements check necessary
+   * and make the movement if required.
+   *
+   * @return block collection ID
+   */
+  public synchronized Long get() {
+    return storageMovementNeeded.poll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/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
new file mode 100644
index 0000000..b5aed37
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
@@ -0,0 +1,397 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.balancer.Matcher;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Setting storagePolicy on a file after the file write will only update the new
+ * storage policy type in Namespace, but physical block storage movement will
+ * not happen until user runs "Mover Tool" explicitly for such files. The
+ * StoragePolicySatisfier Daemon thread implemented for addressing the case
+ * where users may want to physically move the blocks by HDFS itself instead of
+ * running mover tool explicitly. Just calling client API to
+ * satisfyStoragePolicy on a file/dir will automatically trigger to move its
+ * physical storage locations as expected in asynchronous manner. Here Namenode
+ * will pick the file blocks which are expecting to change its storages, then it
+ * will build the mapping of source block location and expected storage type and
+ * location to move. After that this class will also prepare commands to send to
+ * Datanode for processing the physical block movements.
+ */
+@InterfaceAudience.Private
+public class StoragePolicySatisfier implements Runnable {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(StoragePolicySatisfier.class);
+  private Daemon storagePolicySatisfierThread;
+  private final Namesystem namesystem;
+  private final BlockManager blockManager;
+  private final BlockStorageMovementNeeded storageMovementNeeded;
+
+  public StoragePolicySatisfier(final Namesystem namesystem,
+      final BlockStorageMovementNeeded storageMovementNeeded,
+      final BlockManager blkManager) {
+    this.namesystem = namesystem;
+    this.storageMovementNeeded = storageMovementNeeded;
+    this.blockManager = blkManager;
+  }
+
+  /**
+   * Start storage policy satisfier demon thread.
+   */
+  public void start() {
+    storagePolicySatisfierThread = new Daemon(this);
+    storagePolicySatisfierThread.setName("StoragePolicySatisfier");
+    storagePolicySatisfierThread.start();
+  }
+
+  /**
+   * Stop storage policy satisfier demon thread.
+   */
+  public void stop() {
+    if (storagePolicySatisfierThread == null) {
+      return;
+    }
+    storagePolicySatisfierThread.interrupt();
+    try {
+      storagePolicySatisfierThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  @Override
+  public void run() {
+    while (namesystem.isRunning()) {
+      try {
+        Long blockCollectionID = storageMovementNeeded.get();
+        if (blockCollectionID != null) {
+          computeAndAssignStorageMismatchedBlocksToDNs(blockCollectionID);
+        }
+        // TODO: We can think to make this as configurable later, how frequently
+        // we want to check block movements.
+        Thread.sleep(3000);
+      } catch (Throwable t) {
+        if (!namesystem.isRunning()) {
+          LOG.info("Stopping StoragePolicySatisfier.");
+          if (!(t instanceof InterruptedException)) {
+            LOG.info("StoragePolicySatisfier received an exception"
+                + " while shutting down.", t);
+          }
+          break;
+        }
+        LOG.error("StoragePolicySatisfier thread received runtime exception. "
+            + "Stopping Storage policy satisfier work", t);
+        // TODO: Just break for now. Once we implement dynamic start/stop
+        // option, we can add conditions here when to break/terminate.
+        break;
+      }
+    }
+  }
+
+  private void computeAndAssignStorageMismatchedBlocksToDNs(
+      long blockCollectionID) {
+    BlockCollection blockCollection =
+        namesystem.getBlockCollection(blockCollectionID);
+    if (blockCollection == null) {
+      return;
+    }
+    byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
+    BlockStoragePolicy existingStoragePolicy =
+        blockManager.getStoragePolicy(existingStoragePolicyID);
+    if (!blockCollection.getLastBlock().isComplete()) {
+      // Postpone, currently file is under construction
+      // So, should we add back? or leave it to user
+      return;
+    }
+
+    // First datanode will be chosen as the co-ordinator node for storage
+    // movements. Later this can be optimized if needed.
+    DatanodeDescriptor coordinatorNode = null;
+    BlockInfo[] blocks = blockCollection.getBlocks();
+    List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
+    for (int i = 0; i < blocks.length; i++) {
+      BlockInfo blockInfo = blocks[i];
+      List<StorageType> expectedStorageTypes =
+          existingStoragePolicy.chooseStorageTypes(blockInfo.getReplication());
+      DatanodeStorageInfo[] storages = blockManager.getStorages(blockInfo);
+      StorageType[] storageTypes = new StorageType[storages.length];
+      for (int j = 0; j < storages.length; j++) {
+        DatanodeStorageInfo datanodeStorageInfo = storages[j];
+        StorageType storageType = datanodeStorageInfo.getStorageType();
+        storageTypes[j] = storageType;
+      }
+      List<StorageType> existing =
+          new LinkedList<StorageType>(Arrays.asList(storageTypes));
+      if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
+          existing, true)) {
+        List<StorageTypeNodePair> sourceWithStorageMap =
+            new ArrayList<StorageTypeNodePair>();
+        List<DatanodeStorageInfo> existingBlockStorages =
+            new ArrayList<DatanodeStorageInfo>(Arrays.asList(storages));
+        for (StorageType existingType : existing) {
+          Iterator<DatanodeStorageInfo> iterator =
+              existingBlockStorages.iterator();
+          while (iterator.hasNext()) {
+            DatanodeStorageInfo datanodeStorageInfo = iterator.next();
+            StorageType storageType = datanodeStorageInfo.getStorageType();
+            if (storageType == existingType) {
+              iterator.remove();
+              sourceWithStorageMap.add(new StorageTypeNodePair(storageType,
+                  datanodeStorageInfo.getDatanodeDescriptor()));
+              break;
+            }
+          }
+        }
+
+        StorageTypeNodeMap locsForExpectedStorageTypes =
+            findTargetsForExpectedStorageTypes(expectedStorageTypes);
+
+        BlockMovingInfo blockMovingInfo =
+            findSourceAndTargetToMove(blockInfo, existing, sourceWithStorageMap,
+                expectedStorageTypes, locsForExpectedStorageTypes);
+        if (coordinatorNode == null) {
+          // For now, first datanode will be chosen as the co-ordinator. Later
+          // this can be optimized if needed.
+          coordinatorNode =
+              (DatanodeDescriptor) blockMovingInfo.getSources()[0];
+        }
+        blockMovingInfos.add(blockMovingInfo);
+      }
+    }
+
+    if (blockMovingInfos.size() < 1) {
+      // TODO: Major: handle this case. I think we need retry cases to
+      // be implemented. Idea is, if some files are not getting storage movement
+      // chances, then we can just retry limited number of times and exit.
+      return;
+    }
+    coordinatorNode.addBlocksToMoveStorage(blockMovingInfos);
+  }
+
+  /**
+   * Find the good target node for each source node for which block storages was
+   * misplaced.
+   *
+   * @param blockInfo
+   *          - Block
+   * @param existing
+   *          - Existing storage types of block
+   * @param sourceWithStorageList
+   *          - Source Datanode with storages list
+   * @param expected
+   *          - Expecting storages to move
+   * @param locsForExpectedStorageTypes
+   *          - Available DNs for expected storage types
+   * @return list of block source and target node pair
+   */
+  private BlockMovingInfo findSourceAndTargetToMove(BlockInfo blockInfo,
+      List<StorageType> existing,
+      List<StorageTypeNodePair> sourceWithStorageList,
+      List<StorageType> expected,
+      StorageTypeNodeMap locsForExpectedStorageTypes) {
+    List<DatanodeInfo> sourceNodes = new ArrayList<>();
+    List<StorageType> sourceStorageTypes = new ArrayList<>();
+    List<DatanodeInfo> targetNodes = new ArrayList<>();
+    List<StorageType> targetStorageTypes = new ArrayList<>();
+    List<DatanodeDescriptor> chosenNodes = new ArrayList<>();
+    for (int i = 0; i < sourceWithStorageList.size(); i++) {
+      StorageTypeNodePair existingTypeNodePair = sourceWithStorageList.get(i);
+      StorageTypeNodePair chosenTarget =
+          chooseTargetTypeInSameNode(existingTypeNodePair.dn, expected,
+              locsForExpectedStorageTypes, chosenNodes);
+
+      if (chosenTarget == null && blockManager.getDatanodeManager()
+          .getNetworkTopology().isNodeGroupAware()) {
+        chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
+            expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
+            chosenNodes);
+      }
+
+      // Then, match nodes on the same rack
+      if (chosenTarget == null) {
+        chosenTarget =
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
+                Matcher.SAME_RACK, locsForExpectedStorageTypes, chosenNodes);
+      }
+
+      if (chosenTarget == null) {
+        chosenTarget =
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
+                Matcher.ANY_OTHER, locsForExpectedStorageTypes, chosenNodes);
+      }
+      if (null != chosenTarget) {
+        sourceNodes.add(existingTypeNodePair.dn);
+        sourceStorageTypes.add(existingTypeNodePair.storageType);
+        targetNodes.add(chosenTarget.dn);
+        targetStorageTypes.add(chosenTarget.storageType);
+        chosenNodes.add(chosenTarget.dn);
+        // TODO: We can increment scheduled block count for this node?
+      } else {
+        // TODO: Failed to ChooseTargetNodes...So let just retry. Shall we
+        // proceed without this targets? Then what should be final result?
+        // How about pack empty target, means target node could not be chosen ,
+        // so result should be RETRY_REQUIRED from DN always.
+        // Log..unable to choose target node for source datanodeDescriptor
+        sourceNodes.add(existingTypeNodePair.dn);
+        sourceStorageTypes.add(existingTypeNodePair.storageType);
+        targetNodes.add(null);
+        targetStorageTypes.add(null);
+      }
+    }
+    BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blockInfo,
+        sourceNodes.toArray(new DatanodeInfo[sourceNodes.size()]),
+        targetNodes.toArray(new DatanodeInfo[targetNodes.size()]),
+        sourceStorageTypes.toArray(new StorageType[sourceStorageTypes.size()]),
+        targetStorageTypes.toArray(new StorageType[targetStorageTypes.size()]));
+    return blkMovingInfo;
+  }
+
+  /**
+   * Choose the target storage within same Datanode if possible.
+   *
+   * @param locsForExpectedStorageTypes
+   * @param chosenNodes
+   */
+  private StorageTypeNodePair chooseTargetTypeInSameNode(
+      DatanodeDescriptor source, List<StorageType> targetTypes,
+      StorageTypeNodeMap locsForExpectedStorageTypes,
+      List<DatanodeDescriptor> chosenNodes) {
+    for (StorageType t : targetTypes) {
+      DatanodeStorageInfo chooseStorage4Block =
+          source.chooseStorage4Block(t, 0);
+      if (chooseStorage4Block != null) {
+        return new StorageTypeNodePair(t, source);
+      }
+    }
+    return null;
+  }
+
+  private StorageTypeNodePair chooseTarget(Block block,
+      DatanodeDescriptor source, List<StorageType> targetTypes, Matcher matcher,
+      StorageTypeNodeMap locsForExpectedStorageTypes,
+      List<DatanodeDescriptor> chosenNodes) {
+    for (StorageType t : targetTypes) {
+      List<DatanodeDescriptor> nodesWithStorages =
+          locsForExpectedStorageTypes.getNodesWithStorages(t);
+      Collections.shuffle(nodesWithStorages);
+      for (DatanodeDescriptor target : nodesWithStorages) {
+        if (!chosenNodes.contains(target) && matcher.match(
+            blockManager.getDatanodeManager().getNetworkTopology(), source,
+            target)) {
+          if (null != target.chooseStorage4Block(t, block.getNumBytes())) {
+            return new StorageTypeNodePair(t, target);
+          }
+        }
+      }
+    }
+    return null;
+  }
+
+  private static class StorageTypeNodePair {
+    public StorageType storageType = null;
+    public DatanodeDescriptor dn = null;
+
+    public StorageTypeNodePair(StorageType storageType, DatanodeDescriptor dn) {
+      this.storageType = storageType;
+      this.dn = dn;
+    }
+  }
+
+  private StorageTypeNodeMap findTargetsForExpectedStorageTypes(
+      List<StorageType> expected) {
+    StorageTypeNodeMap targetMap = new StorageTypeNodeMap();
+    List<DatanodeDescriptor> reports = blockManager.getDatanodeManager()
+        .getDatanodeListForReport(DatanodeReportType.LIVE);
+    for (DatanodeDescriptor dn : reports) {
+      StorageReport[] storageReports = dn.getStorageReports();
+      for (StorageReport storageReport : storageReports) {
+        StorageType t = storageReport.getStorage().getStorageType();
+        if (expected.contains(t)) {
+          final long maxRemaining = getMaxRemaining(dn.getStorageReports(), t);
+          if (maxRemaining > 0L) {
+            targetMap.add(t, dn);
+          }
+        }
+      }
+    }
+    return targetMap;
+  }
+
+  private static long getMaxRemaining(StorageReport[] storageReports,
+      StorageType t) {
+    long max = 0L;
+    for (StorageReport r : storageReports) {
+      if (r.getStorage().getStorageType() == t) {
+        if (r.getRemaining() > max) {
+          max = r.getRemaining();
+        }
+      }
+    }
+    return max;
+  }
+
+  private static class StorageTypeNodeMap {
+    private final EnumMap<StorageType, List<DatanodeDescriptor>> typeNodeMap =
+        new EnumMap<StorageType, List<DatanodeDescriptor>>(StorageType.class);
+
+    private void add(StorageType t, DatanodeDescriptor dn) {
+      List<DatanodeDescriptor> nodesWithStorages = getNodesWithStorages(t);
+      LinkedList<DatanodeDescriptor> value = null;
+      if (nodesWithStorages == null) {
+        value = new LinkedList<DatanodeDescriptor>();
+        value.add(dn);
+        typeNodeMap.put(t, value);
+      } else {
+        nodesWithStorages.add(dn);
+      }
+    }
+
+    /**
+     * @param type
+     *          - Storage type
+     * @return datanodes which has the given storage type
+     */
+    private List<DatanodeDescriptor> getNodesWithStorages(StorageType type) {
+      return typeNodeMap.get(type);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
index 42ba265..c1ab800 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.server.protocol;
 import java.util.Arrays;
 
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 
 /**
  * A BlockStorageMovementCommand is an instruction to a DataNode to move the
@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
  * NameNode about the movement status.
  */
 public class BlockStorageMovementCommand extends DatanodeCommand {
-
   // TODO: constructor needs to be refined based on the block movement data
   // structure.
   BlockStorageMovementCommand(int action) {
@@ -46,13 +45,13 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
    * Stores block to storage info that can be used for block movement.
    */
   public static class BlockMovingInfo {
-    private ExtendedBlock blk;
+    private Block blk;
     private DatanodeInfo[] sourceNodes;
     private StorageType[] sourceStorageTypes;
     private DatanodeInfo[] targetNodes;
     private StorageType[] targetStorageTypes;
 
-    public BlockMovingInfo(ExtendedBlock block,
+    public BlockMovingInfo(Block block,
         DatanodeInfo[] sourceDnInfos, DatanodeInfo[] targetDnInfos,
         StorageType[] srcStorageTypes, StorageType[] targetStorageTypes) {
       this.blk = block;
@@ -62,11 +61,11 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
       this.targetStorageTypes = targetStorageTypes;
     }
 
-    public void addBlock(ExtendedBlock block) {
+    public void addBlock(Block block) {
       this.blk = block;
     }
 
-    public ExtendedBlock getBlock() {
+    public Block getBlock() {
       return this.blk;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/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 c722306..d803f1a 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
@@ -30,8 +30,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
@@ -71,14 +71,14 @@ public class TestStoragePolicySatisfyWorker {
   public void testMoveSingleBlockToAnotherDatanode() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     initConf(conf);
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(4)
-        .storageTypes(
-            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
-                {StorageType.DISK, StorageType.ARCHIVE},
-                {StorageType.DISK, StorageType.ARCHIVE},
-                {StorageType.DISK, StorageType.ARCHIVE}})
-        .build();
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(4)
+            .storageTypes(
+                new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+                    {StorageType.DISK, StorageType.ARCHIVE},
+                    {StorageType.ARCHIVE, StorageType.ARCHIVE},
+                    {StorageType.ARCHIVE, StorageType.ARCHIVE}})
+            .build();
     try {
       cluster.waitActive();
       final DistributedFileSystem dfs = cluster.getFileSystem();
@@ -108,12 +108,12 @@ public class TestStoragePolicySatisfyWorker {
           src);
       List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
       BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
-          lb.getBlock(), lb.getLocations()[0], targetDnInfo,
+          lb.getBlock().getLocalBlock(), lb.getLocations()[0], targetDnInfo,
           lb.getStorageTypes()[0], StorageType.ARCHIVE);
       blockMovingInfos.add(blockMovingInfo);
       INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
       worker.processBlockMovingTasks(inode.getId(),
-          blockMovingInfos);
+          cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
       cluster.triggerHeartbeats();
 
       // Wait till NameNode notified about the block location details
@@ -150,7 +150,7 @@ public class TestStoragePolicySatisfyWorker {
     }, 100, timeout);
   }
 
-  BlockMovingInfo prepareBlockMovingInfo(ExtendedBlock block,
+  BlockMovingInfo prepareBlockMovingInfo(Block block,
       DatanodeInfo src, DatanodeInfo destin, StorageType storageType,
       StorageType targetStorageType) {
     return new BlockMovingInfo(block, new DatanodeInfo[] {src},

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b52abb/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
new file mode 100644
index 0000000..b61814d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Supplier;
+
+/**
+ * Tests that StoragePolicySatisfier daemon is able to check the blocks to be
+ * moved and finding its suggested target locations to move.
+ */
+public class TestStoragePolicySatisfier {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestStoragePolicySatisfier.class);
+  private final Configuration config = new HdfsConfiguration();
+  private StorageType[][] allDiskTypes =
+      new StorageType[][]{{StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK}};
+  private MiniDFSCluster hdfsCluster = null;
+  final private int numOfDatanodes = 3;
+  final private int storagesPerDatanode = 2;
+  final private long capacity = 2 * 256 * 1024 * 1024;
+  final private String file = "/testMoveWhenStoragePolicyNotSatisfying";
+  private DistributedFileSystem distributedFS = null;
+
+  @Before
+  public void setUp() throws IOException {
+    config.setLong("dfs.block.size", 1024);
+    hdfsCluster = startCluster(config, allDiskTypes, numOfDatanodes,
+        storagesPerDatanode, capacity);
+    distributedFS = hdfsCluster.getFileSystem();
+    writeContent(distributedFS, file);
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToCOLD()
+      throws Exception {
+
+    try {
+      // Change policy to ALL_SSD
+      distributedFS.setStoragePolicy(new Path(file), "COLD");
+      Set<DatanodeDescriptor> previousNodes =
+          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
+              .getDatanodeManager().getDatanodes();
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+
+      hdfsCluster.triggerHeartbeats();
+      // Wait till namenode notified about the block location details
+      waitExpectedStorageType(StorageType.ARCHIVE, distributedFS, previousNodes,
+          6, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToALLSSD()
+      throws Exception {
+    try {
+      // Change policy to ALL_SSD
+      distributedFS.setStoragePolicy(new Path(file), "ALL_SSD");
+      Set<DatanodeDescriptor> previousNodes =
+          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
+              .getDatanodeManager().getDatanodes();
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK},
+              {StorageType.SSD, StorageType.DISK},
+              {StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      waitExpectedStorageType(StorageType.SSD, distributedFS, previousNodes, 6,
+          30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  private void writeContent(final DistributedFileSystem dfs,
+      final String fileName) throws IOException {
+    // write to DISK
+    final FSDataOutputStream out = dfs.create(new Path(fileName));
+    for (int i = 0; i < 1000; i++) {
+      out.writeChars("t");
+    }
+    out.close();
+  }
+
+  private void startAdditionalDNs(final Configuration conf,
+      int newNodesRequired, int existingNodesNum, StorageType[][] newTypes,
+      int storagesPerDatanode, long capacity, final MiniDFSCluster cluster)
+          throws IOException {
+    long[][] capacities;
+    existingNodesNum += newNodesRequired;
+    capacities = new long[newNodesRequired][storagesPerDatanode];
+    for (int i = 0; i < newNodesRequired; i++) {
+      for (int j = 0; j < storagesPerDatanode; j++) {
+        capacities[i][j] = capacity;
+      }
+    }
+
+    cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null,
+        null, capacities, null, false, false, false, null);
+    cluster.triggerHeartbeats();
+  }
+
+  private MiniDFSCluster startCluster(final Configuration conf,
+      StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
+      long nodeCapacity) throws IOException {
+    long[][] capacities = new long[numberOfDatanodes][storagesPerDn];
+    for (int i = 0; i < numberOfDatanodes; i++) {
+      for (int j = 0; j < storagesPerDn; j++) {
+        capacities[i][j] = nodeCapacity;
+      }
+    }
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numberOfDatanodes).storagesPerDatanode(storagesPerDn)
+        .storageTypes(storageTypes).storageCapacities(capacities).build();
+    cluster.waitActive();
+    return cluster;
+  }
+
+  // TODO: this assertion can be changed to end to end based assertion later
+  // when DN side processing work integrated to this work.
+  private void waitExpectedStorageType(final StorageType expectedStorageType,
+      final DistributedFileSystem dfs,
+      final Set<DatanodeDescriptor> previousNodes, int expectedArchiveCount,
+      int timeout) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        Iterator<DatanodeDescriptor> iterator = previousNodes.iterator();
+        int archiveCount = 0;
+        while (iterator.hasNext()) {
+          DatanodeDescriptor dn = iterator.next();
+          List<BlockMovingInfo> pendingItemsToMove =
+              dn.getStorageMovementPendingItems();
+          for (BlockMovingInfo blkInfoToMoveStorage : pendingItemsToMove) {
+            StorageType[] targetStorageTypes =
+                blkInfoToMoveStorage.getTargetStorageTypes();
+            for (StorageType storageType : targetStorageTypes) {
+              if (storageType == expectedStorageType) {
+                archiveCount++;
+              }
+            }
+          }
+        }
+        LOG.info(
+            expectedStorageType + " replica count, expected={} and actual={}",
+            expectedArchiveCount, archiveCount);
+        return expectedArchiveCount == archiveCount;
+      }
+    }, 100, timeout);
+  }
+}


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


[30/50] [abbrv] hadoop git commit: HDFS-11762. [SPS]: Empty files should be ignored in StoragePolicySatisfier. Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-11762. [SPS]: Empty files should be ignored in StoragePolicySatisfier. Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 8feeadba70213061d5c776e2037ac5b9f84bc462
Parents: 8de5ddf
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Mon Jun 5 12:32:41 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:54 2017 +0530

----------------------------------------------------------------------
 .../namenode/FSDirSatisfyStoragePolicyOp.java   | 15 ++++++---
 .../namenode/TestStoragePolicySatisfier.java    | 32 ++++++++++++++++++++
 2 files changed, 42 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8feeadba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
index 81d337f..bd4e5ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java
@@ -51,7 +51,6 @@ final class FSDirSatisfyStoragePolicyOp {
 
     assert fsd.getFSNamesystem().hasWriteLock();
     FSPermissionChecker pc = fsd.getPermissionChecker();
-    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -62,8 +61,11 @@ final class FSDirSatisfyStoragePolicyOp {
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
       }
       XAttr satisfyXAttr = unprotectedSatisfyStoragePolicy(iip, bm, fsd);
-      xAttrs.add(satisfyXAttr);
-      fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+      if (satisfyXAttr != null) {
+        List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+        xAttrs.add(satisfyXAttr);
+        fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+      }
     } finally {
       fsd.writeUnlock();
     }
@@ -79,16 +81,19 @@ final class FSDirSatisfyStoragePolicyOp {
 
     // TODO: think about optimization here, label the dir instead
     // of the sub-files of the dir.
-    if (inode.isFile()) {
+    if (inode.isFile() && inode.asFile().numBlocks() != 0) {
       candidateNodes.add(inode);
     } else if (inode.isDirectory()) {
       for (INode node : inode.asDirectory().getChildrenList(snapshotId)) {
-        if (node.isFile()) {
+        if (node.isFile() && node.asFile().numBlocks() != 0) {
           candidateNodes.add(node);
         }
       }
     }
 
+    if (candidateNodes.isEmpty()) {
+      return null;
+    }
     // If node has satisfy xattr, then stop adding it
     // to satisfy movement queue.
     if (inodeHasSatisfyXAttr(candidateNodes)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8feeadba/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 fa954b8..8e08a1e 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
@@ -907,6 +907,38 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Test SPS with empty file.
+   * 1. Create one empty file.
+   * 2. Call satisfyStoragePolicy for empty file.
+   * 3. SPS should skip this file and xattr should not be added for empty file.
+   */
+  @Test(timeout = 300000)
+  public void testSPSWhenFileLengthIsZero() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(0)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path filePath = new Path("/zeroSizeFile");
+      DFSTestUtil.createFile(fs, filePath, 0, (short) 1, 0);
+      FSEditLog editlog = cluster.getNameNode().getNamesystem().getEditLog();
+      long lastWrittenTxId = editlog.getLastWrittenTxId();
+      fs.satisfyStoragePolicy(filePath);
+      Assert.assertEquals("Xattr should not be added for the file",
+          lastWrittenTxId, editlog.getLastWrittenTxId());
+      INode inode = cluster.getNameNode().getNamesystem().getFSDirectory()
+          .getINode(filePath.toString());
+      Assert.assertTrue("XAttrFeature should be null for file",
+          inode.getXAttrFeature() == null);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
       throws IOException {
     ArrayList<DataNode> dns = hdfsCluster.getDataNodes();


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


[27/50] [abbrv] hadoop git commit: HDFS-11572. [SPS]: SPS should clean Xattrs when no blocks required to satisfy for a file. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11572. [SPS]: SPS should clean Xattrs when no blocks required to satisfy for a file. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: 24c819bbdba583c7cf4fc325f7cb2ed5351880e3
Parents: 32f2375
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Thu Apr 20 23:14:36 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:32 2017 +0530

----------------------------------------------------------------------
 .../BlockStorageMovementAttemptedItems.java     |   2 +-
 .../server/namenode/StoragePolicySatisfier.java | 116 ++++++++++++++-----
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  35 ++++++
 .../TestPersistentStoragePolicySatisfier.java   |  52 +++++----
 .../namenode/TestStoragePolicySatisfier.java    |  76 ++++++++++++
 5 files changed, 225 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/24c819bb/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 f2406da..bf7859c 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
@@ -333,7 +333,7 @@ public class BlockStorageMovementAttemptedItems {
                   + "doesn't exists in storageMovementAttemptedItems list",
                   storageMovementAttemptedResult.getTrackId());
               // Remove xattr for the track id.
-              this.sps.notifyBlkStorageMovementFinished(
+              this.sps.postBlkStorageMovementCleanup(
                   storageMovementAttemptedResult.getTrackId());
             }
             break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24c819bb/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 8be0a2a..3b20314 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
@@ -79,6 +79,27 @@ public class StoragePolicySatisfier implements Runnable {
   private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
   private volatile boolean isRunning = false;
 
+  /**
+   * Represents the collective analysis status for all blocks.
+   */
+  private enum BlocksMovingAnalysisStatus {
+    // Represents that, the analysis skipped due to some conditions. A such
+    // condition is if block collection is in incomplete state.
+    ANALYSIS_SKIPPED_FOR_RETRY,
+    // Represents that, all block storage movement needed blocks found its
+    // targets.
+    ALL_BLOCKS_TARGETS_PAIRED,
+    // Represents that, only fewer or none of the block storage movement needed
+    // block found its eligible targets.
+    FEW_BLOCKS_TARGETS_PAIRED,
+    // Represents that, none of the blocks found for block storage movements.
+    BLOCKS_ALREADY_SATISFIED,
+    // Represents that, the analysis skipped due to some conditions.
+    // Example conditions are if no blocks really exists in block collection or
+    // if analysis is not required on ec files with unsuitable storage policies
+    BLOCKS_TARGET_PAIRING_SKIPPED;
+  }
+
   public StoragePolicySatisfier(final Namesystem namesystem,
       final BlockStorageMovementNeeded storageMovementNeeded,
       final BlockManager blkManager, Configuration conf) {
@@ -208,10 +229,31 @@ public class StoragePolicySatisfier implements Runnable {
                 namesystem.getBlockCollection(blockCollectionID);
             // Check blockCollectionId existence.
             if (blockCollection != null) {
-              boolean allBlockLocsAttemptedToSatisfy =
-                  computeAndAssignStorageMismatchedBlocksToDNs(blockCollection);
-              this.storageMovementsMonitor
-                  .add(blockCollectionID, allBlockLocsAttemptedToSatisfy);
+              BlocksMovingAnalysisStatus status =
+                  analyseBlocksStorageMovementsAndAssignToDN(blockCollection);
+              switch (status) {
+              // Just add to monitor, so it will be retried after timeout
+              case ANALYSIS_SKIPPED_FOR_RETRY:
+                // Just add to monitor, so it will be tracked for result and
+                // be removed on successful storage movement result.
+              case ALL_BLOCKS_TARGETS_PAIRED:
+                this.storageMovementsMonitor.add(blockCollectionID, true);
+                break;
+              // Add to monitor with allBlcoksAttemptedToSatisfy flag false, so
+              // that it will be tracked and still it will be consider for retry
+              // as analysis was not found targets for storage movement blocks.
+              case FEW_BLOCKS_TARGETS_PAIRED:
+                this.storageMovementsMonitor.add(blockCollectionID, false);
+                break;
+              // Just clean Xattrs
+              case BLOCKS_TARGET_PAIRING_SKIPPED:
+              case BLOCKS_ALREADY_SATISFIED:
+              default:
+                LOG.info("Block analysis skipped or blocks already satisfied"
+                    + " with storages. So, Cleaning up the Xattrs.");
+                postBlkStorageMovementCleanup(blockCollectionID);
+                break;
+              }
             }
           }
         }
@@ -235,15 +277,15 @@ public class StoragePolicySatisfier implements Runnable {
         }
         LOG.error("StoragePolicySatisfier thread received runtime exception. "
             + "Stopping Storage policy satisfier work", t);
-        // TODO: Just break for now. Once we implement dynamic start/stop
-        // option, we can add conditions here when to break/terminate.
         break;
       }
     }
   }
 
-  private boolean computeAndAssignStorageMismatchedBlocksToDNs(
+  private BlocksMovingAnalysisStatus analyseBlocksStorageMovementsAndAssignToDN(
       BlockCollection blockCollection) {
+    BlocksMovingAnalysisStatus status =
+        BlocksMovingAnalysisStatus.BLOCKS_ALREADY_SATISFIED;
     byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
     BlockStoragePolicy existingStoragePolicy =
         blockManager.getStoragePolicy(existingStoragePolicyID);
@@ -252,21 +294,20 @@ public class StoragePolicySatisfier implements Runnable {
       // So, should we add back? or leave it to user
       LOG.info("BlockCollectionID: {} file is under construction. So, postpone"
           + " this to the next retry iteration", blockCollection.getId());
-      return true;
+      return BlocksMovingAnalysisStatus.ANALYSIS_SKIPPED_FOR_RETRY;
     }
 
     // First datanode will be chosen as the co-ordinator node for storage
     // movements. Later this can be optimized if needed.
     DatanodeDescriptor coordinatorNode = null;
     BlockInfo[] blocks = blockCollection.getBlocks();
+    if (blocks.length == 0) {
+      LOG.info("BlockCollectionID: {} file is not having any blocks."
+          + " So, skipping the analysis.", blockCollection.getId());
+      return BlocksMovingAnalysisStatus.BLOCKS_TARGET_PAIRING_SKIPPED;
+    }
     List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
 
-    // True value represents that, SPS is able to find matching target nodes
-    // to satisfy storage type for all the blocks locations of the given
-    // blockCollection. A false value represents that, blockCollection needed
-    // retries to satisfy the storage policy for some of the block locations.
-    boolean foundMatchingTargetNodesForAllBlocks = true;
-
     for (int i = 0; i < blocks.length; i++) {
       BlockInfo blockInfo = blocks[i];
       List<StorageType> expectedStorageTypes;
@@ -283,19 +324,38 @@ public class StoragePolicySatisfier implements Runnable {
           LOG.warn("The storage policy " + existingStoragePolicy.getName()
               + " is not suitable for Striped EC files. "
               + "So, ignoring to move the blocks");
-          return false;
+          return BlocksMovingAnalysisStatus.BLOCKS_TARGET_PAIRING_SKIPPED;
         }
       } else {
         expectedStorageTypes = existingStoragePolicy
             .chooseStorageTypes(blockInfo.getReplication());
       }
-      foundMatchingTargetNodesForAllBlocks |= computeBlockMovingInfos(
-          blockMovingInfos, blockInfo, expectedStorageTypes);
+
+      DatanodeStorageInfo[] storages = blockManager.getStorages(blockInfo);
+      StorageType[] storageTypes = new StorageType[storages.length];
+      for (int j = 0; j < storages.length; j++) {
+        DatanodeStorageInfo datanodeStorageInfo = storages[j];
+        StorageType storageType = datanodeStorageInfo.getStorageType();
+        storageTypes[j] = storageType;
+      }
+      List<StorageType> existing =
+          new LinkedList<StorageType>(Arrays.asList(storageTypes));
+      if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
+          existing, true)) {
+        boolean computeStatus = computeBlockMovingInfos(blockMovingInfos,
+            blockInfo, expectedStorageTypes, existing, storages);
+        if (computeStatus
+            && status != BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED) {
+          status = BlocksMovingAnalysisStatus.ALL_BLOCKS_TARGETS_PAIRED;
+        } else {
+          status = BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED;
+        }
+      }
     }
 
     assignBlockMovingInfosToCoordinatorDn(blockCollection.getId(),
         blockMovingInfos, coordinatorNode);
-    return foundMatchingTargetNodesForAllBlocks;
+    return status;
   }
 
   /**
@@ -311,22 +371,18 @@ public class StoragePolicySatisfier implements Runnable {
    *          - block details
    * @param expectedStorageTypes
    *          - list of expected storage type to satisfy the storage policy
+   * @param existing
+   *          - list to get existing storage types
+   * @param storages
+   *          - available storages
    * @return false if some of the block locations failed to find target node to
    *         satisfy the storage policy, true otherwise
    */
   private boolean computeBlockMovingInfos(
       List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
-      List<StorageType> expectedStorageTypes) {
+      List<StorageType> expectedStorageTypes, List<StorageType> existing,
+      DatanodeStorageInfo[] storages) {
     boolean foundMatchingTargetNodesForBlock = true;
-    DatanodeStorageInfo[] storages = blockManager.getStorages(blockInfo);
-    StorageType[] storageTypes = new StorageType[storages.length];
-    for (int j = 0; j < storages.length; j++) {
-      DatanodeStorageInfo datanodeStorageInfo = storages[j];
-      StorageType storageType = datanodeStorageInfo.getStorageType();
-      storageTypes[j] = storageType;
-    }
-    List<StorageType> existing =
-        new LinkedList<StorageType>(Arrays.asList(storageTypes));
     if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
         existing, true)) {
       List<StorageTypeNodePair> sourceWithStorageMap =
@@ -756,7 +812,7 @@ public class StoragePolicySatisfier implements Runnable {
     Long id;
     while ((id = storageMovementNeeded.get()) != null) {
       try {
-        notifyBlkStorageMovementFinished(id);
+        postBlkStorageMovementCleanup(id);
       } catch (IOException ie) {
         LOG.warn("Failed to remove SPS "
             + "xattr for collection id " + id, ie);
@@ -771,7 +827,7 @@ public class StoragePolicySatisfier implements Runnable {
    * @param trackId track id i.e., block collection id.
    * @throws IOException
    */
-  public void notifyBlkStorageMovementFinished(long trackId)
+  public void postBlkStorageMovementCleanup(long trackId)
       throws IOException {
     this.namesystem.getFSDirectory().removeSPSXattr(trackId);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24c819bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 1cec9b1..255454c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -98,8 +98,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
@@ -153,9 +156,12 @@ import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.namenode.XAttrStorage;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -173,6 +179,7 @@ import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -2391,4 +2398,32 @@ public class DFSTestUtil {
       }
     }, 500, timeout);
   }
+
+  /**
+   * Waits for removal of a specified Xattr on a specified file.
+   *
+   * @param srcPath
+   *          file name.
+   * @param xattr
+   *          name of the extended attribute.
+   * @param ns
+   *          Namesystem
+   * @param timeout
+   *          max wait time
+   * @throws Exception
+   */
+  public static void waitForXattrRemoved(String srcPath, String xattr,
+      Namesystem ns, int timeout) throws TimeoutException, InterruptedException,
+          UnresolvedLinkException, AccessControlException,
+          ParentNotDirectoryException {
+    final INode inode = ns.getFSDirectory().getINode(srcPath);
+    final XAttr satisfyXAttr = XAttrHelper.buildXAttr(xattr);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
+        return !existingXAttrs.contains(satisfyXAttr);
+      }
+    }, 100, timeout);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24c819bb/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 8c3359a..41c272c 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
@@ -20,22 +20,18 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.List;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
-import static org.junit.Assert.assertFalse;
 
 /**
  * Test persistence of satisfying files/directories.
@@ -341,15 +337,9 @@ public class TestPersistentStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           testFileName, StorageType.DISK, 2, timeout, fs);
 
-      // Make sure that SPS xattr has been removed.
-      int retryTime = 0;
-      while (retryTime < 30) {
-        if (!fileContainsSPSXAttr(testFile)) {
-          break;
-        }
-        Thread.sleep(minCheckTimeout);
-        retryTime += 1;
-      }
+      // Make sure satisfy xattr has been removed.
+      DFSTestUtil.waitForXattrRemoved(testFileName,
+          XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
 
       fs.setStoragePolicy(testFile, COLD);
       fs.satisfyStoragePolicy(testFile);
@@ -379,7 +369,8 @@ public class TestPersistentStoragePolicySatisfier {
       cluster.getNamesystem().getBlockManager().deactivateSPS();
 
       // Make sure satisfy xattr has been removed.
-      assertFalse(fileContainsSPSXAttr(testFile));
+      DFSTestUtil.waitForXattrRemoved(testFileName,
+          XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
 
     } finally {
       clusterShutdown();
@@ -387,18 +378,29 @@ public class TestPersistentStoragePolicySatisfier {
   }
 
   /**
-   * Check whether file contains SPS xattr.
-   * @param fileName file name.
-   * @return true if file contains SPS xattr.
-   * @throws IOException
+   * Tests that Xattrs should be cleaned if all blocks already satisfied.
+   *
+   * @throws Exception
    */
-  private boolean fileContainsSPSXAttr(Path fileName) throws IOException {
-    final INode inode = cluster.getNamesystem()
-        .getFSDirectory().getINode(fileName.toString());
-    final XAttr satisfyXAttr =
-        XAttrHelper.buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
-    List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
-    return existingXAttrs.contains(satisfyXAttr);
+  @Test(timeout = 300000)
+  public void testSPSShouldNotLeakXattrIfStorageAlreadySatisfied()
+      throws Exception {
+    try {
+      clusterSetUp();
+      DFSTestUtil.waitExpectedStorageType(testFileName, StorageType.DISK, 3,
+          timeout, fs);
+      fs.satisfyStoragePolicy(testFile);
+
+      DFSTestUtil.waitExpectedStorageType(testFileName, StorageType.DISK, 3,
+          timeout, fs);
+
+      // Make sure satisfy xattr has been removed.
+      DFSTestUtil.waitForXattrRemoved(testFileName,
+          XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
+
+    } finally {
+      clusterShutdown();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24c819bb/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 2a33455..8457e5b 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 import static org.junit.Assert.assertNull;
 
 import java.io.FileNotFoundException;
@@ -34,13 +35,17 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -828,6 +833,77 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Tests that Xattrs should be cleaned if satisfy storage policy called on EC
+   * file with unsuitable storage policy set.
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testSPSShouldNotLeakXattrIfSatisfyStoragePolicyCallOnECFiles()
+      throws Exception {
+    StorageType[][] diskTypes =
+        new StorageType[][]{{StorageType.SSD, StorageType.DISK},
+            {StorageType.SSD, StorageType.DISK},
+            {StorageType.SSD, StorageType.DISK},
+            {StorageType.SSD, StorageType.DISK},
+            {StorageType.SSD, StorageType.DISK},
+            {StorageType.DISK, StorageType.SSD},
+            {StorageType.DISK, StorageType.SSD},
+            {StorageType.DISK, StorageType.SSD},
+            {StorageType.DISK, StorageType.SSD},
+            {StorageType.DISK, StorageType.SSD}};
+
+    int defaultStripedBlockSize =
+        ErasureCodingPolicyManager.getSystemPolicies()[0].getCellSize() * 4;
+    config.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultStripedBlockSize);
+    config.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    config.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
+        1L);
+    config.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
+        false);
+
+    try {
+      hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
+          storagesPerDatanode, capacity);
+
+      // set "/foo" directory with ONE_SSD storage policy.
+      ClientProtocol client = NameNodeProxies.createProxy(config,
+          hdfsCluster.getFileSystem(0).getUri(), ClientProtocol.class)
+          .getProxy();
+      String fooDir = "/foo";
+      client.mkdirs(fooDir, new FsPermission((short) 777), true);
+      // set an EC policy on "/foo" directory
+      client.setErasureCodingPolicy(fooDir, null);
+
+      // write file to fooDir
+      final String testFile = "/foo/bar";
+      long fileLen = 20 * defaultStripedBlockSize;
+      dfs = hdfsCluster.getFileSystem();
+      DFSTestUtil.createFile(dfs, new Path(testFile), fileLen, (short) 3, 0);
+
+      // ONESSD is unsuitable storage policy on EC files
+      client.setStoragePolicy(fooDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+      dfs.satisfyStoragePolicy(new Path(testFile));
+
+      // Thread.sleep(9000); // To make sure SPS triggered
+      // verify storage types and locations
+      LocatedBlocks locatedBlocks =
+          client.getBlockLocations(testFile, 0, fileLen);
+      for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
+        for (StorageType type : lb.getStorageTypes()) {
+          Assert.assertEquals(StorageType.DISK, type);
+        }
+      }
+
+      // Make sure satisfy xattr has been removed.
+      DFSTestUtil.waitForXattrRemoved(testFile, XATTR_SATISFY_STORAGE_POLICY,
+          hdfsCluster.getNamesystem(), 30000);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
       throws IOException {
     ArrayList<DataNode> dns = hdfsCluster.getDataNodes();


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


[49/50] [abbrv] hadoop git commit: HDFS-12790: [SPS]: Rebasing HDFS-10285 branch after HDFS-10467, HDFS-12599 and HDFS-11968 commits. Contributed by Rakesh R.

Posted by ra...@apache.org.
HDFS-12790: [SPS]: Rebasing HDFS-10285 branch after HDFS-10467, HDFS-12599 and HDFS-11968 commits. 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/bd059a1c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bd059a1c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bd059a1c

Branch: refs/heads/HDFS-10285
Commit: bd059a1cde2ad6549cd4222e230115dab013417e
Parents: ee0e8b0
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Nov 10 10:06:43 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:31:29 2017 +0530

----------------------------------------------------------------------
 .../federation/router/RouterRpcServer.java      |  19 +++
 .../namenode/TestStoragePolicySatisfier.java    |   9 +-
 ...stStoragePolicySatisfierWithStripedFile.java |  21 +--
 .../hdfs/tools/TestStoragePolicyCommands.java   |  57 ---------
 .../TestStoragePolicySatisfyAdminCommands.java  | 127 +++++++++++++++++++
 5 files changed, 162 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd059a1c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index b5acf12..f395f51 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -84,6 +84,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -2062,4 +2063,22 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     UserGroupInformation ugi = Server.getRemoteUser();
     return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
   }
+
+  @Override
+  public void satisfyStoragePolicy(String path) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public boolean isStoragePolicySatisfierRunning() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return false;
+  }
+
+  @Override
+  public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
+      String path) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return StoragePolicySatisfyPathStatus.NOT_AVAILABLE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd059a1c/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 f42d911..edd1aca 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
@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.InternalDataNodeTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.Assert;
@@ -912,8 +913,6 @@ public class TestStoragePolicySatisfier {
 
     int defaultStripedBlockSize =
         StripedFileTestUtil.getDefaultECPolicy().getCellSize() * 4;
-    config.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
-        StripedFileTestUtil.getDefaultECPolicy().getName());
     config.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultStripedBlockSize);
     config.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
     config.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
@@ -925,6 +924,9 @@ public class TestStoragePolicySatisfier {
     try {
       hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
           storagesPerDatanode, capacity);
+      dfs = hdfsCluster.getFileSystem();
+      dfs.enableErasureCodingPolicy(
+          StripedFileTestUtil.getDefaultECPolicy().getName());
 
       // set "/foo" directory with ONE_SSD storage policy.
       ClientProtocol client = NameNodeProxies.createProxy(config,
@@ -939,7 +941,6 @@ public class TestStoragePolicySatisfier {
       // write file to fooDir
       final String testFile = "/foo/bar";
       long fileLen = 20 * defaultStripedBlockSize;
-      dfs = hdfsCluster.getFileSystem();
       DFSTestUtil.createFile(dfs, new Path(testFile), fileLen, (short) 3, 0);
 
       // ONESSD is unsuitable storage policy on EC files
@@ -1632,7 +1633,7 @@ public class TestStoragePolicySatisfier {
       LOG.info("Simulate block pinning in datanode {}",
           locations[favoredNodesCount]);
       DataNode dn = hdfsCluster.getDataNode(dnInfo.getIpcPort());
-      DataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
+      InternalDataNodeTestUtils.mockDatanodeBlkPinning(dn, true);
       favoredNodesCount--;
       if (favoredNodesCount <= 0) {
         break; // marked favoredNodesCount number of pinned block location

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd059a1c/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 154ddae..6991ad2 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
@@ -105,8 +105,6 @@ 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);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -128,6 +126,9 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      dfs.enableErasureCodingPolicy(
+          StripedFileTestUtil.getDefaultECPolicy().getName());
 
       // set "/bar" directory with HOT storage policy.
       ClientProtocol client = NameNodeProxies.createProxy(conf,
@@ -215,8 +216,6 @@ 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);
@@ -240,7 +239,9 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
-
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      dfs.enableErasureCodingPolicy(
+          StripedFileTestUtil.getDefaultECPolicy().getName());
       // set "/bar" directory with HOT storage policy.
       ClientProtocol client = NameNodeProxies.createProxy(conf,
           cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
@@ -327,8 +328,6 @@ public class TestStoragePolicySatisfierWithStripedFile {
     conf.set(DFSConfigKeys
         .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         "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);
@@ -350,6 +349,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     try {
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
+      fs.enableErasureCodingPolicy(
+          StripedFileTestUtil.getDefaultECPolicy().getName());
       Path barDir = new Path("/bar");
       fs.mkdirs(barDir);
       // set an EC policy on "/bar" directory
@@ -419,8 +420,6 @@ 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);
@@ -444,7 +443,9 @@ public class TestStoragePolicySatisfierWithStripedFile {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     try {
       cluster.waitActive();
-
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      dfs.enableErasureCodingPolicy(
+          StripedFileTestUtil.getDefaultECPolicy().getName());
       // set "/bar" directory with HOT storage policy.
       ClientProtocol client = NameNodeProxies.createProxy(conf,
           cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd059a1c/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 0644a83..7e0663d 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
@@ -165,61 +165,4 @@ public class TestStoragePolicyCommands {
     DFSTestUtil.toolRun(admin, "-getStoragePolicy -path /fooz", 2,
         "File/Directory does not exist: /fooz");
   }
-
-  @Test(timeout = 30000)
-  public void testStoragePolicySatisfierCommand() throws Exception {
-    final String file = "/testStoragePolicySatisfierCommand";
-    DFSTestUtil.createFile(fs, new Path(file), SIZE, REPL, 0);
-
-    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
-    DFSTestUtil.toolRun(admin, "-getStoragePolicy -path " + file, 0,
-        "The storage policy of " + file + " is unspecified");
-
-    DFSTestUtil.toolRun(admin,
-        "-setStoragePolicy -path " + file + " -policy COLD", 0,
-        "Set storage policy COLD on " + file.toString());
-
-    DFSTestUtil.toolRun(admin, "-satisfyStoragePolicy -path " + file, 0,
-        "Scheduled blocks to move based on the current storage policy on "
-            + file.toString());
-
-    DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
-        fs);
-  }
-
-  @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, "-isSatisfierRunning", 0, "yes");
-
-    cluster.getNameNode().reconfigureProperty(
-        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
-    cluster.waitActive();
-
-    DFSTestUtil.toolRun(admin, "-isSatisfierRunning", 0, "no");
-
-    // Test with unnecessary args
-    DFSTestUtil.toolRun(admin, "-isSatisfierRunning status", 1,
-        "Can't understand arguments: ");
-  }
-
-  @Test(timeout = 90000)
-  public void testSatisfyStoragePolicyCommandWithWaitOption()
-      throws Exception {
-    final String file = "/testSatisfyStoragePolicyCommandWithWaitOption";
-    DFSTestUtil.createFile(fs, new Path(file), SIZE, REPL, 0);
-
-    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
-
-    DFSTestUtil.toolRun(admin, "-setStoragePolicy -path " + file
-        + " -policy COLD", 0, "Set storage policy COLD on " + file.toString());
-
-    DFSTestUtil.toolRun(admin, "-satisfyStoragePolicy -w -path " + file, 0,
-        "Waiting for satisfy the policy");
-
-    DFSTestUtil
-        .waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000, fs);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd059a1c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicySatisfyAdminCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicySatisfyAdminCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicySatisfyAdminCommands.java
new file mode 100644
index 0000000..856c3ec
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicySatisfyAdminCommands.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test StoragePolicySatisfy admin commands.
+ */
+public class TestStoragePolicySatisfyAdminCommands {
+  private static final short REPL = 1;
+  private static final int SIZE = 128;
+
+  private Configuration conf = null;
+  private MiniDFSCluster cluster = null;
+  private DistributedFileSystem dfs = null;
+
+  @Before
+  public void clusterSetUp() throws IOException, URISyntaxException {
+    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)
+        .storageTypes(newtypes).build();
+    cluster.waitActive();
+    dfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void clusterShutdown() throws IOException{
+    if(dfs != null) {
+      dfs.close();
+      dfs = null;
+    }
+    if(cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  @Test(timeout = 30000)
+  public void testStoragePolicySatisfierCommand() throws Exception {
+    final String file = "/testStoragePolicySatisfierCommand";
+    DFSTestUtil.createFile(dfs, new Path(file), SIZE, REPL, 0);
+
+    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
+    DFSTestUtil.toolRun(admin, "-getStoragePolicy -path " + file, 0,
+        "The storage policy of " + file + " is unspecified");
+
+    DFSTestUtil.toolRun(admin,
+        "-setStoragePolicy -path " + file + " -policy COLD", 0,
+        "Set storage policy COLD on " + file.toString());
+
+    DFSTestUtil.toolRun(admin, "-satisfyStoragePolicy -path " + file, 0,
+        "Scheduled blocks to move based on the current storage policy on "
+            + file.toString());
+
+    DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
+        dfs);
+  }
+
+  @Test(timeout = 30000)
+  public void testIsSatisfierRunningCommand() throws Exception {
+    final String file = "/testIsSatisfierRunningCommand";
+    DFSTestUtil.createFile(dfs, new Path(file), SIZE, REPL, 0);
+    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
+    DFSTestUtil.toolRun(admin, "-isSatisfierRunning", 0, "yes");
+
+    cluster.getNameNode().reconfigureProperty(
+        DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY, "false");
+    cluster.waitActive();
+
+    DFSTestUtil.toolRun(admin, "-isSatisfierRunning", 0, "no");
+
+    // Test with unnecessary args
+    DFSTestUtil.toolRun(admin, "-isSatisfierRunning status", 1,
+        "Can't understand arguments: ");
+  }
+
+  @Test(timeout = 90000)
+  public void testSatisfyStoragePolicyCommandWithWaitOption()
+      throws Exception {
+    final String file = "/testSatisfyStoragePolicyCommandWithWaitOption";
+    DFSTestUtil.createFile(dfs, new Path(file), SIZE, REPL, 0);
+
+    final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
+
+    DFSTestUtil.toolRun(admin, "-setStoragePolicy -path " + file
+        + " -policy COLD", 0, "Set storage policy COLD on " + file.toString());
+
+    DFSTestUtil.toolRun(admin, "-satisfyStoragePolicy -w -path " + file, 0,
+        "Waiting for satisfy the policy");
+
+    DFSTestUtil.waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000,
+        dfs);
+  }
+}


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


[26/50] [abbrv] hadoop git commit: HDFS-11334: [SPS]: NN switch and rescheduling movements can lead to have more than one coordinator for same file blocks. Contributed by Rakesh R.

Posted by ra...@apache.org.
HDFS-11334: [SPS]: NN switch and rescheduling movements can lead to have more than one coordinator for same file blocks. 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/32f2375a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/32f2375a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/32f2375a

Branch: refs/heads/HDFS-10285
Commit: 32f2375ade95a8ed0cc6017aff2084137b761e7b
Parents: af15803
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Tue Apr 18 15:23:58 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:27 2017 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   6 +
 .../server/blockmanagement/DatanodeManager.java |  12 ++
 .../hdfs/server/datanode/BPServiceActor.java    |   4 +-
 .../datanode/BlockStorageMovementTracker.java   |  37 +++-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  12 +-
 .../datanode/StoragePolicySatisfyWorker.java    |  95 +++++++++--
 .../BlockStorageMovementAttemptedItems.java     |  80 ++++++---
 .../server/namenode/StoragePolicySatisfier.java |  15 +-
 .../protocol/BlocksStorageMovementResult.java   |   6 +-
 .../src/main/proto/DatanodeProtocol.proto       |   1 +
 .../TestStoragePolicySatisfyWorker.java         |  68 ++++----
 .../TestStoragePolicySatisfierWithHA.java       | 170 +++++++++++++++++--
 13 files changed, 413 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/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 6b5b69b..fcac7a5 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
@@ -581,7 +581,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY =
       "dfs.storage.policy.satisfier.self.retry.timeout.millis";
   public static final int DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT =
-      30 * 60 * 1000;
+      20 * 60 * 1000;
 
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 9866;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 3b38077..5044c0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -981,6 +981,9 @@ public class PBHelper {
       case FAILURE:
         status = Status.FAILURE;
         break;
+      case IN_PROGRESS:
+        status = Status.IN_PROGRESS;
+        break;
       default:
         throw new AssertionError("Unknown status: " + resultProto.getStatus());
       }
@@ -1007,6 +1010,9 @@ public class PBHelper {
       case FAILURE:
         status = BlocksStorageMovementResultProto.Status.FAILURE;
         break;
+      case IN_PROGRESS:
+        status = BlocksStorageMovementResultProto.Status.IN_PROGRESS;
+        break;
       default:
         throw new AssertionError("Unknown status: " + report.getStatus());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 51c5aef..a298843 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1089,6 +1089,18 @@ public class DatanodeManager {
           nodeS.setSoftwareVersion(nodeReg.getSoftwareVersion());
           nodeS.setDisallowed(false); // Node is in the include list
 
+          // Sets dropSPSWork flag to true, to ensure that
+          // DNA_DROP_SPS_WORK_COMMAND will send to datanode via next heartbeat
+          // response immediately after the node registration. This is
+          // to avoid a situation, where multiple trackId responses coming from
+          // different co-odinator datanodes. After SPS monitor time out, it
+          // will retry the files which were scheduled to the disconnected(for
+          // long time more than heartbeat expiry) DN, by finding new
+          // co-ordinator datanode. Now, if the expired datanode reconnects back
+          // after SPS reschedules, it leads to get different movement results
+          // from reconnected and new DN co-ordinators.
+          nodeS.setDropSPSWork(true);
+
           // resolve network location
           if(this.rejectUnresolvedTopologyDN) {
             nodeS.setNetworkLocation(resolveNetworkLocation(nodeS));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index 0f93fb0..f537f49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -536,7 +536,7 @@ class BPServiceActor implements Runnable {
 
     // Remove the blocks movement results after successfully transferring
     // to namenode.
-    dn.getStoragePolicySatisfyWorker().getBlocksMovementsCompletionHandler()
+    dn.getStoragePolicySatisfyWorker().getBlocksMovementsStatusHandler()
         .remove(blksMovementResults);
 
     return response;
@@ -544,7 +544,7 @@ class BPServiceActor implements Runnable {
 
   private BlocksStorageMovementResult[] getBlocksMovementResults() {
     List<BlocksStorageMovementResult> trackIdVsMovementStatus = dn
-        .getStoragePolicySatisfyWorker().getBlocksMovementsCompletionHandler()
+        .getStoragePolicySatisfyWorker().getBlocksMovementsStatusHandler()
         .getBlksMovementResults();
     BlocksStorageMovementResult[] blksMovementResult =
         new BlocksStorageMovementResult[trackIdVsMovementStatus.size()];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
index e623cef..99858bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockStorageMovementTracker.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -28,7 +29,7 @@ import java.util.concurrent.Future;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlockMovementResult;
-import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlocksMovementsCompletionHandler;
+import org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker.BlocksMovementsStatusHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,32 +42,34 @@ public class BlockStorageMovementTracker implements Runnable {
   private static final Logger LOG = LoggerFactory
       .getLogger(BlockStorageMovementTracker.class);
   private final CompletionService<BlockMovementResult> moverCompletionService;
-  private final BlocksMovementsCompletionHandler blksMovementscompletionHandler;
+  private final BlocksMovementsStatusHandler blksMovementsStatusHandler;
 
   // Keeps the information - trackID vs its list of blocks
   private final Map<Long, List<Future<BlockMovementResult>>> moverTaskFutures;
   private final Map<Long, List<BlockMovementResult>> movementResults;
 
+  private volatile boolean running = true;
+
   /**
    * BlockStorageMovementTracker constructor.
    *
    * @param moverCompletionService
    *          completion service.
    * @param handler
-   *          blocks movements completion handler
+   *          blocks movements status handler
    */
   public BlockStorageMovementTracker(
       CompletionService<BlockMovementResult> moverCompletionService,
-      BlocksMovementsCompletionHandler handler) {
+      BlocksMovementsStatusHandler handler) {
     this.moverCompletionService = moverCompletionService;
     this.moverTaskFutures = new HashMap<>();
-    this.blksMovementscompletionHandler = handler;
+    this.blksMovementsStatusHandler = handler;
     this.movementResults = new HashMap<>();
   }
 
   @Override
   public void run() {
-    while (true) {
+    while (running) {
       if (moverTaskFutures.size() <= 0) {
         try {
           synchronized (moverTaskFutures) {
@@ -95,8 +98,8 @@ public class BlockStorageMovementTracker implements Runnable {
             synchronized (moverTaskFutures) {
               moverTaskFutures.remove(trackId);
             }
-            // handle completed blocks movements per trackId.
-            blksMovementscompletionHandler.handle(resultPerTrackIdList);
+            // handle completed or inprogress blocks movements per trackId.
+            blksMovementsStatusHandler.handle(resultPerTrackIdList);
             movementResults.remove(trackId);
           }
         }
@@ -158,4 +161,22 @@ public class BlockStorageMovementTracker implements Runnable {
       movementResults.clear();
     }
   }
+
+  /**
+   * @return the list of trackIds which are still waiting to complete all the
+   *         scheduled blocks movements.
+   */
+  Set<Long> getInProgressTrackIds() {
+    synchronized (moverTaskFutures) {
+      return moverTaskFutures.keySet();
+    }
+  }
+
+  /**
+   * Sets running flag to false and clear the pending movement result queues.
+   */
+  public void stopTracking() {
+    running = false;
+    removeAll();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 9a8de99..11755cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1423,6 +1423,7 @@ public class DataNode extends ReconfigurableBase
     blockRecoveryWorker = new BlockRecoveryWorker(this);
     storagePolicySatisfyWorker =
         new StoragePolicySatisfyWorker(getConf(), this);
+    storagePolicySatisfyWorker.start();
 
     blockPoolManager = new BlockPoolManager(this);
     blockPoolManager.refreshNamenodes(getConf());
@@ -1968,7 +1969,11 @@ public class DataNode extends ReconfigurableBase
         }
       }
     }
-    
+
+    // stop storagePolicySatisfyWorker
+    if (storagePolicySatisfyWorker != null) {
+      storagePolicySatisfyWorker.stop();
+    }
     List<BPOfferService> bposArray = (this.blockPoolManager == null)
         ? new ArrayList<BPOfferService>()
         : this.blockPoolManager.getAllNamenodeThreads();
@@ -2121,6 +2126,11 @@ public class DataNode extends ReconfigurableBase
       notifyAll();
     }
     tracer.close();
+
+    // Waiting to finish SPS worker thread.
+    if (storagePolicySatisfyWorker != null) {
+      storagePolicySatisfyWorker.waitToFinishWorkerThread();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index a96ac98..f4f97dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -31,7 +32,9 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
@@ -87,10 +90,13 @@ public class StoragePolicySatisfyWorker {
   private final int moverThreads;
   private final ExecutorService moveExecutor;
   private final CompletionService<BlockMovementResult> moverCompletionService;
-  private final BlocksMovementsCompletionHandler handler;
+  private final BlocksMovementsStatusHandler handler;
   private final BlockStorageMovementTracker movementTracker;
   private Daemon movementTrackerThread;
 
+  private long inprogressTrackIdsCheckInterval = 30 * 1000; // 30seconds.
+  private long nextInprogressRecheckTime;
+
   public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
     this.datanode = datanode;
     this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
@@ -99,15 +105,52 @@ public class StoragePolicySatisfyWorker {
         DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
     moveExecutor = initializeBlockMoverThreadPool(moverThreads);
     moverCompletionService = new ExecutorCompletionService<>(moveExecutor);
-    handler = new BlocksMovementsCompletionHandler();
+    handler = new BlocksMovementsStatusHandler();
     movementTracker = new BlockStorageMovementTracker(moverCompletionService,
         handler);
     movementTrackerThread = new Daemon(movementTracker);
     movementTrackerThread.setName("BlockStorageMovementTracker");
-    movementTrackerThread.start();
+
+    // Interval to check that the inprogress trackIds. The time interval is
+    // proportional o the heart beat interval time period.
+    final long heartbeatIntervalSeconds = conf.getTimeDuration(
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
+    inprogressTrackIdsCheckInterval = 5 * heartbeatIntervalSeconds;
+    // update first inprogress recheck time to a future time stamp.
+    nextInprogressRecheckTime = monotonicNow()
+        + inprogressTrackIdsCheckInterval;
+
     // TODO: Needs to manage the number of concurrent moves per DataNode.
   }
 
+  /**
+   * Start StoragePolicySatisfyWorker, which will start block movement tracker
+   * thread to track the completion of block movements.
+   */
+  void start() {
+    movementTrackerThread.start();
+  }
+
+  /**
+   * Stop StoragePolicySatisfyWorker, which will stop block movement tracker
+   * thread.
+   */
+  void stop() {
+    movementTrackerThread.interrupt();
+    movementTracker.stopTracking();
+  }
+
+  /**
+   * Timed wait to stop BlockStorageMovement tracker daemon thread.
+   */
+  void waitToFinishWorkerThread() {
+    try {
+      movementTrackerThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
   private ThreadPoolExecutor initializeBlockMoverThreadPool(int num) {
     LOG.debug("Block mover to satisfy storage policy; pool threads={}", num);
 
@@ -352,11 +395,11 @@ public class StoragePolicySatisfyWorker {
   }
 
   /**
-   * Blocks movements completion handler, which is used to collect details of
-   * the completed list of block movements and this status(success or failure)
-   * will be send to the namenode via heartbeat.
+   * Blocks movements status handler, which is used to collect details of the
+   * completed or inprogress list of block movements and this status(success or
+   * failure or inprogress) will be send to the namenode via heartbeat.
    */
-  static class BlocksMovementsCompletionHandler {
+  class BlocksMovementsStatusHandler {
     private final List<BlocksStorageMovementResult> trackIdVsMovementStatus =
         new ArrayList<>();
 
@@ -395,14 +438,21 @@ public class StoragePolicySatisfyWorker {
      * @return unmodifiable list of blocks storage movement results.
      */
     List<BlocksStorageMovementResult> getBlksMovementResults() {
+      List<BlocksStorageMovementResult> movementResults = new ArrayList<>();
+      // 1. Adding all the completed trackids.
       synchronized (trackIdVsMovementStatus) {
-        if (trackIdVsMovementStatus.size() <= 0) {
-          return new ArrayList<>();
+        if (trackIdVsMovementStatus.size() > 0) {
+          movementResults = Collections
+              .unmodifiableList(trackIdVsMovementStatus);
         }
-        List<BlocksStorageMovementResult> results = Collections
-            .unmodifiableList(trackIdVsMovementStatus);
-        return results;
       }
+      // 2. Adding the in progress track ids after those which are completed.
+      Set<Long> inProgressTrackIds = getInProgressTrackIds();
+      for (Long trackId : inProgressTrackIds) {
+        movementResults.add(new BlocksStorageMovementResult(trackId,
+            BlocksStorageMovementResult.Status.IN_PROGRESS));
+      }
+      return movementResults;
     }
 
     /**
@@ -433,7 +483,7 @@ public class StoragePolicySatisfyWorker {
   }
 
   @VisibleForTesting
-  BlocksMovementsCompletionHandler getBlocksMovementsCompletionHandler() {
+  BlocksMovementsStatusHandler getBlocksMovementsStatusHandler() {
     return handler;
   }
 
@@ -447,4 +497,23 @@ public class StoragePolicySatisfyWorker {
     movementTracker.removeAll();
     handler.removeAll();
   }
+
+  /**
+   * Gets list of trackids which are inprogress. Will do collection periodically
+   * on 'dfs.datanode.storage.policy.satisfier.worker.inprogress.recheck.time.
+   * millis' interval.
+   *
+   * @return collection of trackids which are inprogress
+   */
+  private Set<Long> getInProgressTrackIds() {
+    Set<Long> trackIds = new HashSet<>();
+    long now = monotonicNow();
+    if (nextInprogressRecheckTime >= now) {
+      trackIds = movementTracker.getInProgressTrackIds();
+
+      // schedule next re-check interval
+      nextInprogressRecheckTime = now + inprogressTrackIdsCheckInterval;
+    }
+    return trackIds;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/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 26b98d8..f2406da 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
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult.Status;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,9 +40,11 @@ import com.google.common.annotations.VisibleForTesting;
  * A monitor class for checking whether block storage movements finished or not.
  * If block storage movement results from datanode indicates about the movement
  * success, then it will just remove the entries from tracking. If it reports
- * failure, then it will add back to needed block storage movements list. If no
- * DN reports about movement for longer time, then such items will be retries
- * automatically after timeout. The default timeout would be 30mins.
+ * failure, then it will add back to needed block storage movements list. If it
+ * reports in_progress, that means the blocks movement is in progress and the
+ * coordinator is still tracking the movement. If no DN reports about movement
+ * for longer time, then such items will be retries automatically after timeout.
+ * The default timeout would be 30mins.
  */
 public class BlockStorageMovementAttemptedItems {
   private static final Logger LOG =
@@ -57,10 +60,10 @@ public class BlockStorageMovementAttemptedItems {
   private Daemon timerThread = null;
   private final StoragePolicySatisfier sps;
   //
-  // It might take anywhere between 30 to 60 minutes before
+  // It might take anywhere between 20 to 60 minutes before
   // a request is timed out.
   //
-  private long selfRetryTimeout = 30 * 60 * 1000;
+  private long selfRetryTimeout = 20 * 60 * 1000;
 
   //
   // It might take anywhere between 5 to 10 minutes before
@@ -159,35 +162,35 @@ public class BlockStorageMovementAttemptedItems {
 
   /**
    * This class contains information of an attempted trackID. Information such
-   * as, (a)last attempted time stamp, (b)whether all the blocks in the trackID
-   * were attempted and blocks movement has been scheduled to satisfy storage
-   * policy. This is used by
+   * as, (a)last attempted or reported time stamp, (b)whether all the blocks in
+   * the trackID were attempted and blocks movement has been scheduled to
+   * satisfy storage policy. This is used by
    * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
    */
   private final static class ItemInfo {
-    private final long lastAttemptedTimeStamp;
+    private long lastAttemptedOrReportedTime;
     private final boolean allBlockLocsAttemptedToSatisfy;
 
     /**
      * ItemInfo constructor.
      *
-     * @param lastAttemptedTimeStamp
-     *          last attempted time stamp
+     * @param lastAttemptedOrReportedTime
+     *          last attempted or reported time
      * @param allBlockLocsAttemptedToSatisfy
      *          whether all the blocks in the trackID were attempted and blocks
      *          movement has been scheduled to satisfy storage policy
      */
-    private ItemInfo(long lastAttemptedTimeStamp,
+    private ItemInfo(long lastAttemptedOrReportedTime,
         boolean allBlockLocsAttemptedToSatisfy) {
-      this.lastAttemptedTimeStamp = lastAttemptedTimeStamp;
+      this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
       this.allBlockLocsAttemptedToSatisfy = allBlockLocsAttemptedToSatisfy;
     }
 
     /**
-     * @return last attempted time stamp.
+     * @return last attempted or reported time stamp.
      */
-    private long getLastAttemptedTimeStamp() {
-      return lastAttemptedTimeStamp;
+    private long getLastAttemptedOrReportedTime() {
+      return lastAttemptedOrReportedTime;
     }
 
     /**
@@ -200,6 +203,14 @@ public class BlockStorageMovementAttemptedItems {
     private boolean isAllBlockLocsAttemptedToSatisfy() {
       return allBlockLocsAttemptedToSatisfy;
     }
+
+    /**
+     * Update lastAttemptedOrReportedTime, so that the expiration time will be
+     * postponed to future.
+     */
+    private void touchLastReportedTimeStamp() {
+      this.lastAttemptedOrReportedTime = monotonicNow();
+    }
   }
 
   /**
@@ -234,7 +245,8 @@ public class BlockStorageMovementAttemptedItems {
       while (iter.hasNext()) {
         Entry<Long, ItemInfo> entry = iter.next();
         ItemInfo itemInfo = entry.getValue();
-        if (now > itemInfo.getLastAttemptedTimeStamp() + selfRetryTimeout) {
+        if (now > itemInfo.getLastAttemptedOrReportedTime()
+            + selfRetryTimeout) {
           Long blockCollectionID = entry.getKey();
           synchronized (storageMovementAttemptedResults) {
             if (!isExistInResult(blockCollectionID)) {
@@ -273,6 +285,7 @@ public class BlockStorageMovementAttemptedItems {
       Iterator<BlocksStorageMovementResult> resultsIter =
           storageMovementAttemptedResults.iterator();
       while (resultsIter.hasNext()) {
+        boolean isInprogress = false;
         // TrackID need to be retried in the following cases:
         // 1) All or few scheduled block(s) movement has been failed.
         // 2) All the scheduled block(s) movement has been succeeded but there
@@ -282,16 +295,19 @@ public class BlockStorageMovementAttemptedItems {
         BlocksStorageMovementResult storageMovementAttemptedResult = resultsIter
             .next();
         synchronized (storageMovementAttemptedItems) {
-          if (storageMovementAttemptedResult
-              .getStatus() == BlocksStorageMovementResult.Status.FAILURE) {
+          Status status = storageMovementAttemptedResult.getStatus();
+          ItemInfo itemInfo;
+          switch (status) {
+          case FAILURE:
             blockStorageMovementNeeded
                 .add(storageMovementAttemptedResult.getTrackId());
             LOG.warn("Blocks storage movement results for the tracking id: {}"
                 + " is reported from co-ordinating datanode, but result"
                 + " status is FAILURE. So, added for retry",
                 storageMovementAttemptedResult.getTrackId());
-          } else {
-            ItemInfo itemInfo = storageMovementAttemptedItems
+            break;
+          case SUCCESS:
+            itemInfo = storageMovementAttemptedItems
                 .get(storageMovementAttemptedResult.getTrackId());
 
             // ItemInfo could be null. One case is, before the blocks movements
@@ -320,10 +336,26 @@ public class BlockStorageMovementAttemptedItems {
               this.sps.notifyBlkStorageMovementFinished(
                   storageMovementAttemptedResult.getTrackId());
             }
+            break;
+          case IN_PROGRESS:
+            isInprogress = true;
+            itemInfo = storageMovementAttemptedItems
+                .get(storageMovementAttemptedResult.getTrackId());
+            if(itemInfo != null){
+              // update the attempted expiration time to next cycle.
+              itemInfo.touchLastReportedTimeStamp();
+            }
+            break;
+          default:
+            LOG.error("Unknown status: {}", status);
+            break;
+          }
+          // Remove trackID from the attempted list if the attempt has been
+          // completed(success or failure), if any.
+          if (!isInprogress) {
+            storageMovementAttemptedItems
+                .remove(storageMovementAttemptedResult.getTrackId());
           }
-          // Remove trackID from the attempted list, if any.
-          storageMovementAttemptedItems
-              .remove(storageMovementAttemptedResult.getTrackId());
         }
         // Remove trackID from results as processed above.
         resultsIter.remove();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/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 8cf9920..8be0a2a 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
@@ -108,6 +108,11 @@ public class StoragePolicySatisfier implements Runnable {
     } else {
       LOG.info("Starting StoragePolicySatisfier.");
     }
+
+    // Ensure that all the previously submitted block movements(if any) have to
+    // be stopped in all datanodes.
+    addDropSPSWorkCommandsToAllDNs();
+
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
@@ -133,7 +138,7 @@ public class StoragePolicySatisfier implements Runnable {
       LOG.info("Stopping StoragePolicySatisfier, as admin requested to "
           + "deactivate it.");
       this.clearQueuesWithNotification();
-      this.blockManager.getDatanodeManager().addDropSPSWorkCommandsToAllDNs();
+      addDropSPSWorkCommandsToAllDNs();
     } else {
       LOG.info("Stopping StoragePolicySatisfier.");
     }
@@ -170,6 +175,14 @@ public class StoragePolicySatisfier implements Runnable {
     return namesystem.isFileOpenedForWrite(moverId);
   }
 
+  /**
+   * Adding drop commands to all datanodes to stop performing the satisfier
+   * block movements, if any.
+   */
+  private void addDropSPSWorkCommandsToAllDNs() {
+    this.blockManager.getDatanodeManager().addDropSPSWorkCommandsToAllDNs();
+  }
+
   @Override
   public void run() {
     boolean isMoverRunning = !checkIfMoverRunning();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/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 713b83b..b484eb1 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
@@ -35,9 +35,13 @@ public class BlocksStorageMovementResult {
    * retry these failed blocks movements. Example selected target node is no
    * more running or no space. So, retrying by selecting new target node might
    * work.
+   *
+   * <p>
+   * IN_PROGRESS - If all or some of the blocks associated to track id are
+   * still moving.
    */
   public static enum Status {
-    SUCCESS, FAILURE;
+    SUCCESS, FAILURE, IN_PROGRESS;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 899dc7e..080f7fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -192,6 +192,7 @@ message BlocksStorageMovementResultProto {
   enum Status {
     SUCCESS = 1; // block movement succeeded
     FAILURE = 2; // block movement failed and needs to retry
+    IN_PROGRESS = 3; // block movement is still in progress
   }
   required uint64 trackID = 1;
   required Status status = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/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 86b8b50..8fbbf33 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
@@ -176,16 +176,21 @@ public class TestStoragePolicySatisfyWorker {
 
     StoragePolicySatisfyWorker worker = new StoragePolicySatisfyWorker(conf,
         src);
-    List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
-    BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
-        lb.getBlock().getLocalBlock(), lb.getLocations()[0], targetDnInfo,
-        lb.getStorageTypes()[0], StorageType.ARCHIVE);
-    blockMovingInfos.add(blockMovingInfo);
-    INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
-    worker.processBlockMovingTasks(inode.getId(),
-        cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
-
-    waitForBlockMovementCompletion(worker, inode.getId(), 1, 30000);
+    try {
+      worker.start();
+      List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+      BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
+          lb.getBlock().getLocalBlock(), lb.getLocations()[0], targetDnInfo,
+          lb.getStorageTypes()[0], StorageType.ARCHIVE);
+      blockMovingInfos.add(blockMovingInfo);
+      INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
+      worker.processBlockMovingTasks(inode.getId(),
+          cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
+
+      waitForBlockMovementCompletion(worker, inode.getId(), 1, 30000);
+    } finally {
+      worker.stop();
+    }
   }
 
   /**
@@ -212,24 +217,29 @@ public class TestStoragePolicySatisfyWorker {
 
     StoragePolicySatisfyWorker worker =
         new StoragePolicySatisfyWorker(conf, src);
-    List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
-    List<LocatedBlock> locatedBlocks =
-        dfs.getClient().getLocatedBlocks(file, 0).getLocatedBlocks();
-    for (LocatedBlock locatedBlock : locatedBlocks) {
-      BlockMovingInfo blockMovingInfo =
-          prepareBlockMovingInfo(locatedBlock.getBlock().getLocalBlock(),
-              locatedBlock.getLocations()[0], targetDnInfo,
-              locatedBlock.getStorageTypes()[0], StorageType.ARCHIVE);
-      blockMovingInfos.add(blockMovingInfo);
+    worker.start();
+    try {
+      List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+      List<LocatedBlock> locatedBlocks =
+          dfs.getClient().getLocatedBlocks(file, 0).getLocatedBlocks();
+      for (LocatedBlock locatedBlock : locatedBlocks) {
+        BlockMovingInfo blockMovingInfo =
+            prepareBlockMovingInfo(locatedBlock.getBlock().getLocalBlock(),
+                locatedBlock.getLocations()[0], targetDnInfo,
+                locatedBlock.getStorageTypes()[0], StorageType.ARCHIVE);
+        blockMovingInfos.add(blockMovingInfo);
+      }
+      INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
+      worker.processBlockMovingTasks(inode.getId(),
+          cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
+      // Wait till results queue build up
+      waitForBlockMovementResult(worker, inode.getId(), 30000);
+      worker.dropSPSWork();
+      assertTrue(worker.getBlocksMovementsStatusHandler()
+          .getBlksMovementResults().size() == 0);
+    } finally {
+      worker.stop();
     }
-    INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
-    worker.processBlockMovingTasks(inode.getId(),
-        cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
-    // Wait till results queue build up
-    waitForBlockMovementResult(worker, inode.getId(), 30000);
-    worker.dropSPSWork();
-    assertTrue(worker.getBlocksMovementsCompletionHandler()
-        .getBlksMovementResults().size() == 0);
   }
 
   private void waitForBlockMovementResult(
@@ -239,7 +249,7 @@ public class TestStoragePolicySatisfyWorker {
       @Override
       public Boolean get() {
         List<BlocksStorageMovementResult> completedBlocks = worker
-            .getBlocksMovementsCompletionHandler().getBlksMovementResults();
+            .getBlocksMovementsStatusHandler().getBlksMovementResults();
         return completedBlocks.size() > 0;
       }
     }, 100, timeout);
@@ -252,7 +262,7 @@ public class TestStoragePolicySatisfyWorker {
       @Override
       public Boolean get() {
         List<BlocksStorageMovementResult> completedBlocks = worker
-            .getBlocksMovementsCompletionHandler().getBlksMovementResults();
+            .getBlocksMovementsStatusHandler().getBlksMovementResults();
         int failedCount = 0;
         for (BlocksStorageMovementResult blkMovementResult : completedBlocks) {
           if (blkMovementResult.getStatus() ==

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32f2375a/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 4d226ff..c88d5be 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
@@ -17,51 +17,90 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.util.Time.monotonicNow;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationException;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
-
-import java.io.IOException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Tests that StoragePolicySatisfier is able to work with HA enabled.
  */
 public class TestStoragePolicySatisfierWithHA {
   private MiniDFSCluster cluster = null;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestStoragePolicySatisfierWithHA.class);
 
-  @Before
-  public void setUp() throws IOException {
-    Configuration conf = new Configuration();
+  private final Configuration config = new HdfsConfiguration();
+  private static final int DEFAULT_BLOCK_SIZE = 1024;
+  private DistributedFileSystem dfs = null;
+
+  private StorageType[][] allDiskTypes =
+      new StorageType[][]{{StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK}};
+  private int numOfDatanodes = 3;
+  private int storagesPerDatanode = 2;
+  private long capacity = 2 * 256 * 1024 * 1024;
+  private int nnIndex = 0;
+
+  private void createCluster() throws IOException {
+    config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    startCluster(config, allDiskTypes, numOfDatanodes, storagesPerDatanode,
+        capacity);
+    dfs = cluster.getFileSystem(nnIndex);
+  }
+
+  private void startCluster(final Configuration conf,
+      StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
+      long nodeCapacity) throws IOException {
+    long[][] capacities = new long[numberOfDatanodes][storagesPerDn];
+    for (int i = 0; i < numberOfDatanodes; i++) {
+      for (int j = 0; j < storagesPerDn; j++) {
+        capacities[i][j] = nodeCapacity;
+      }
+    }
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
-        .numDataNodes(1)
-        .build();
+        .numDataNodes(numberOfDatanodes).storagesPerDatanode(storagesPerDn)
+        .storageTypes(storageTypes).storageCapacities(capacities).build();
+    cluster.waitActive();
+    cluster.transitionToActive(0);
   }
 
   /**
    * Tests to verify that SPS should run/stop automatically when NN state
    * changes between Standby and Active.
    */
-  @Test(timeout = 100000)
+  @Test(timeout = 90000)
   public void testWhenNNHAStateChanges() throws IOException {
     try {
-      DistributedFileSystem fs;
+      createCluster();
       boolean running;
 
-      cluster.waitActive();
-      fs = cluster.getFileSystem(0);
+      dfs = cluster.getFileSystem(1);
 
       try {
-        fs.getClient().isStoragePolicySatisfierRunning();
+        dfs.getClient().isStoragePolicySatisfierRunning();
         Assert.fail("Call this function to Standby NN should "
             + "raise an exception.");
       } catch (RemoteException e) {
@@ -72,14 +111,15 @@ public class TestStoragePolicySatisfierWithHA {
       }
 
       cluster.transitionToActive(0);
-      running = fs.getClient().isStoragePolicySatisfierRunning();
+      dfs = cluster.getFileSystem(0);
+      running = dfs.getClient().isStoragePolicySatisfierRunning();
       Assert.assertTrue("StoragePolicySatisfier should be active "
           + "when NN transits from Standby to Active mode.", running);
 
       // NN transits from Active to Standby
       cluster.transitionToStandby(0);
       try {
-        fs.getClient().isStoragePolicySatisfierRunning();
+        dfs.getClient().isStoragePolicySatisfierRunning();
         Assert.fail("NN in Standby again, call this function should "
             + "raise an exception.");
       } catch (RemoteException e) {
@@ -106,4 +146,104 @@ public class TestStoragePolicySatisfierWithHA {
       cluster.shutdown();
     }
   }
+
+  /**
+   * Test to verify that during namenode switch over will add
+   * DNA_DROP_SPS_WORK_COMMAND to all the datanodes. Later, this will ensure to
+   * drop all the SPS queues at datanode.
+   */
+  @Test(timeout = 90000)
+  public void testNamenodeSwitchoverShouldDropSPSWork() throws Exception {
+    try {
+      createCluster();
+
+      FSNamesystem fsn = cluster.getNamesystem(0);
+      ArrayList<DataNode> dataNodes = cluster.getDataNodes();
+      List<DatanodeDescriptor> listOfDns = new ArrayList<>();
+      for (DataNode dn : dataNodes) {
+        DatanodeDescriptor dnd = NameNodeAdapter.getDatanode(fsn,
+            dn.getDatanodeId());
+        listOfDns.add(dnd);
+      }
+      cluster.shutdownDataNodes();
+
+      cluster.transitionToStandby(0);
+      LOG.info("**Transition to Active**");
+      cluster.transitionToActive(1);
+
+      // Verify that Standby-to-Active transition should set drop SPS flag to
+      // true. This will ensure that DNA_DROP_SPS_WORK_COMMAND will be
+      // propagated to datanode during heartbeat response.
+      int retries = 20;
+      boolean dropSPSWork = false;
+      while (retries > 0) {
+        for (DatanodeDescriptor dnd : listOfDns) {
+          dropSPSWork = dnd.shouldDropSPSWork();
+          if (!dropSPSWork) {
+            retries--;
+            Thread.sleep(250);
+            break;
+          }
+        }
+        if (dropSPSWork) {
+          break;
+        }
+      }
+      Assert.assertTrue("Didn't drop SPS work", dropSPSWork);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test to verify that SPS work will be dropped once the datanode is marked as
+   * expired. Internally 'dropSPSWork' flag is set as true while expiration and
+   * at the time of reconnection, will send DNA_DROP_SPS_WORK_COMMAND to that
+   * datanode.
+   */
+  @Test(timeout = 90000)
+  public void testDeadDatanode() throws Exception {
+    int heartbeatExpireInterval = 2 * 2000;
+    config.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+        3000);
+    config.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1000L);
+    createCluster();
+
+    DataNode dn = cluster.getDataNodes().get(0);
+    DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
+
+    FSNamesystem fsn = cluster.getNamesystem(0);
+    DatanodeDescriptor dnd = NameNodeAdapter.getDatanode(fsn,
+        dn.getDatanodeId());
+    boolean isDead = false;
+    int retries = 20;
+    while (retries > 0) {
+      isDead = dnd.getLastUpdateMonotonic() < (monotonicNow()
+          - heartbeatExpireInterval);
+      if (isDead) {
+        break;
+      }
+      retries--;
+      Thread.sleep(250);
+    }
+    Assert.assertTrue("Datanode is alive", isDead);
+    // Disable datanode heartbeat, so that the datanode will get expired after
+    // the recheck interval and become dead.
+    DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
+
+    // Verify that datanode expiration will set drop SPS flag to
+    // true. This will ensure that DNA_DROP_SPS_WORK_COMMAND will be
+    // propagated to datanode during reconnection.
+    boolean dropSPSWork = false;
+    retries = 50;
+    while (retries > 0) {
+      dropSPSWork = dnd.shouldDropSPSWork();
+      if (dropSPSWork) {
+        break;
+      }
+      retries--;
+      Thread.sleep(100);
+    }
+    Assert.assertTrue("Didn't drop SPS work", dropSPSWork);
+  }
 }


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


[35/50] [abbrv] hadoop git commit: HDFS-11264: [SPS]: Double checks to ensure that SPS/Mover are not running together. Contributed by Rakesh R.

Posted by ra...@apache.org.
HDFS-11264: [SPS]: Double checks to ensure that SPS/Mover are not running together. 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/85ae3a5e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/85ae3a5e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/85ae3a5e

Branch: refs/heads/HDFS-10285
Commit: 85ae3a5e46bf0d46b019a8b4285b8cf9cf0c55ae
Parents: fd26f71
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Jul 12 17:56:56 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:24 2017 +0530

----------------------------------------------------------------------
 .../server/namenode/StoragePolicySatisfier.java | 53 +++++++++++---------
 .../namenode/TestStoragePolicySatisfier.java    |  3 +-
 ...stStoragePolicySatisfierWithStripedFile.java |  5 +-
 3 files changed, 34 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/85ae3a5e/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 97cbf1b..00b4cd0 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
@@ -128,6 +128,14 @@ public class StoragePolicySatisfier implements Runnable {
    */
   public synchronized void start(boolean reconfigStart) {
     isRunning = true;
+    if (checkIfMoverRunning()) {
+      isRunning = false;
+      LOG.error(
+          "Stopping StoragePolicySatisfier thread " + "as Mover ID file "
+              + HdfsServerConstants.MOVER_ID_PATH.toString()
+              + " been opened. Maybe a Mover instance is running!");
+      return;
+    }
     if (reconfigStart) {
       LOG.info("Starting StoragePolicySatisfier, as admin requested to "
           + "activate it.");
@@ -211,20 +219,6 @@ public class StoragePolicySatisfier implements Runnable {
 
   @Override
   public void run() {
-    boolean isMoverRunning = !checkIfMoverRunning();
-    synchronized (this) {
-      isRunning = isMoverRunning;
-      if (!isRunning) {
-        // Stopping monitor thread and clearing queues as well
-        this.clearQueues();
-        this.storageMovementsMonitor.stopGracefully();
-        LOG.error(
-            "Stopping StoragePolicySatisfier thread " + "as Mover ID file "
-                + HdfsServerConstants.MOVER_ID_PATH.toString()
-                + " been opened. Maybe a Mover instance is running!");
-        return;
-      }
-    }
     while (namesystem.isRunning() && isRunning) {
       try {
         if (!namesystem.isInSafeMode()) {
@@ -274,25 +268,34 @@ public class StoragePolicySatisfier implements Runnable {
         // we want to check block movements.
         Thread.sleep(3000);
       } catch (Throwable t) {
-        synchronized (this) {
+        handleException(t);
+      }
+    }
+  }
+
+  private void handleException(Throwable t) {
+    // double check to avoid entering into synchronized block.
+    if (isRunning) {
+      synchronized (this) {
+        if (isRunning) {
           isRunning = false;
           // Stopping monitor thread and clearing queues as well
           this.clearQueues();
           this.storageMovementsMonitor.stopGracefully();
-        }
-        if (!namesystem.isRunning()) {
-          LOG.info("Stopping StoragePolicySatisfier.");
-          if (!(t instanceof InterruptedException)) {
-            LOG.info("StoragePolicySatisfier received an exception"
-                + " while shutting down.", t);
+          if (!namesystem.isRunning()) {
+            LOG.info("Stopping StoragePolicySatisfier.");
+            if (!(t instanceof InterruptedException)) {
+              LOG.info("StoragePolicySatisfier received an exception"
+                  + " while shutting down.", t);
+            }
+            return;
           }
-          break;
         }
-        LOG.error("StoragePolicySatisfier thread received runtime exception. "
-            + "Stopping Storage policy satisfier work", t);
-        break;
       }
     }
+    LOG.error("StoragePolicySatisfier thread received runtime exception. "
+        + "Stopping Storage policy satisfier work", t);
+    return;
   }
 
   private BlocksMovingAnalysisStatus analyseBlocksStorageMovementsAndAssignToDN(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85ae3a5e/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 7127895..be7236b 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
@@ -927,7 +927,8 @@ public class TestStoragePolicySatisfier {
       String fooDir = "/foo";
       client.mkdirs(fooDir, new FsPermission((short) 777), true);
       // set an EC policy on "/foo" directory
-      client.setErasureCodingPolicy(fooDir, null);
+      client.setErasureCodingPolicy(fooDir,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
 
       // write file to fooDir
       final String testFile = "/foo/bar";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85ae3a5e/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 195c9e3..f905ead 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
@@ -323,6 +323,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     conf.set(DFSConfigKeys
         .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
         "3000");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        StripedFileTestUtil.getDefaultECPolicy().getName());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -346,7 +348,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
       Path barDir = new Path("/bar");
       fs.mkdirs(barDir);
       // set an EC policy on "/bar" directory
-      fs.setErasureCodingPolicy(barDir, null);
+      fs.setErasureCodingPolicy(barDir,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
 
       // write file to barDir
       final Path fooFile = new Path("/bar/foo");


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


[03/50] [abbrv] hadoop git commit: HDFS-10794. [SPS]: Provide storage policy satisfy worker at DN for co-ordinating the block storage movement work. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-10794. [SPS]: Provide storage policy satisfy worker at DN for co-ordinating the block storage movement work. 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/6c118eab
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6c118eab
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6c118eab

Branch: refs/heads/HDFS-10285
Commit: 6c118eab5c0e30586c1b9bca13606924e94b0f27
Parents: a2edc4c
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 14 17:02:11 2016 +0800
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:11:41 2017 +0530

----------------------------------------------------------------------
 .../datanode/StoragePolicySatisfyWorker.java    | 258 +++++++++++++++++++
 .../protocol/BlockStorageMovementCommand.java   | 101 ++++++++
 .../TestStoragePolicySatisfyWorker.java         | 160 ++++++++++++
 3 files changed, 519 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c118eab/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
new file mode 100644
index 0000000..6df4e81
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -0,0 +1,258 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * StoragePolicySatisfyWorker handles the storage policy satisfier commands.
+ * These commands would be issued from NameNode as part of Datanode's heart beat
+ * response. BPOfferService delegates the work to this class for handling
+ * BlockStorageMovement commands.
+ */
+@InterfaceAudience.Private
+public class StoragePolicySatisfyWorker {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(StoragePolicySatisfyWorker.class);
+
+  private final DataNode datanode;
+  private final int ioFileBufferSize;
+
+  private final int moverThreads;
+  private final ExecutorService moveExecutor;
+  private final CompletionService<Void> moverExecutorCompletionService;
+  private final List<Future<Void>> moverTaskFutures;
+
+  public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
+    this.datanode = datanode;
+    this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
+
+    moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
+        DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
+    moveExecutor = initializeBlockMoverThreadPool(moverThreads);
+    moverExecutorCompletionService = new ExecutorCompletionService<>(
+        moveExecutor);
+    moverTaskFutures = new ArrayList<>();
+    // TODO: Needs to manage the number of concurrent moves per DataNode.
+  }
+
+  private ThreadPoolExecutor initializeBlockMoverThreadPool(int num) {
+    LOG.debug("Block mover to satisfy storage policy; pool threads={}", num);
+
+    ThreadPoolExecutor moverThreadPool = new ThreadPoolExecutor(1, num, 60,
+        TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+        new Daemon.DaemonFactory() {
+          private final AtomicInteger threadIndex = new AtomicInteger(0);
+
+          @Override
+          public Thread newThread(Runnable r) {
+            Thread t = super.newThread(r);
+            t.setName("BlockMoverTask-" + threadIndex.getAndIncrement());
+            return t;
+          }
+        }, new ThreadPoolExecutor.CallerRunsPolicy() {
+          @Override
+          public void rejectedExecution(Runnable runnable,
+              ThreadPoolExecutor e) {
+            LOG.info("Execution for block movement to satisfy storage policy"
+                + " got rejected, Executing in current thread");
+            // will run in the current thread.
+            super.rejectedExecution(runnable, e);
+          }
+        });
+
+    moverThreadPool.allowCoreThreadTimeOut(true);
+    return moverThreadPool;
+  }
+
+  public void processBlockMovingTasks(long trackID,
+      List<BlockMovingInfo> blockMovingInfos) {
+    Future<Void> moveCallable = null;
+    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
+      assert blkMovingInfo
+          .getSources().length == blkMovingInfo.getTargets().length;
+
+      for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
+        BlockMovingTask blockMovingTask =
+            new BlockMovingTask(blkMovingInfo.getBlock(),
+            blkMovingInfo.getSources()[i],
+            blkMovingInfo.getTargets()[i],
+            blkMovingInfo.getTargetStorageTypes()[i]);
+        moveCallable = moverExecutorCompletionService
+            .submit(blockMovingTask);
+        moverTaskFutures.add(moveCallable);
+      }
+    }
+
+    // TODO: Presently this function act as a blocking call, this has to be
+    // refined by moving the tracking logic to another tracker thread.
+    for (int i = 0; i < moverTaskFutures.size(); i++) {
+      try {
+        moveCallable = moverExecutorCompletionService.take();
+        moveCallable.get();
+      } catch (InterruptedException | ExecutionException e) {
+        // TODO: Failure retries and report back the error to NameNode.
+        LOG.error("Exception while moving block replica to target storage type",
+            e);
+      }
+    }
+  }
+
+  /**
+   * This class encapsulates the process of moving the block replica to the
+   * given target.
+   */
+  private class BlockMovingTask implements Callable<Void> {
+    private final ExtendedBlock block;
+    private final DatanodeInfo source;
+    private final DatanodeInfo target;
+    private final StorageType targetStorageType;
+
+    BlockMovingTask(ExtendedBlock block, DatanodeInfo source,
+        DatanodeInfo target, StorageType targetStorageType) {
+      this.block = block;
+      this.source = source;
+      this.target = target;
+      this.targetStorageType = targetStorageType;
+    }
+
+    @Override
+    public Void call() {
+      moveBlock();
+      return null;
+    }
+
+    private void moveBlock() {
+      LOG.info("Start moving block {}", block);
+
+      LOG.debug("Start moving block:{} from src:{} to destin:{} to satisfy "
+          + "storageType:{}", block, source, target, targetStorageType);
+      Socket sock = null;
+      DataOutputStream out = null;
+      DataInputStream in = null;
+      try {
+        DNConf dnConf = datanode.getDnConf();
+        String dnAddr = target.getXferAddr(dnConf.getConnectToDnViaHostname());
+        sock = datanode.newSocket();
+        NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr),
+            dnConf.getSocketTimeout());
+        sock.setSoTimeout(2 * dnConf.getSocketTimeout());
+        LOG.debug("Connecting to datanode {}", dnAddr);
+
+        OutputStream unbufOut = sock.getOutputStream();
+        InputStream unbufIn = sock.getInputStream();
+
+        Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
+            block, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
+
+        DataEncryptionKeyFactory keyFactory = datanode
+            .getDataEncryptionKeyFactoryForBlock(block);
+        IOStreamPair saslStreams = datanode.getSaslClient().socketSend(sock,
+            unbufOut, unbufIn, keyFactory, accessToken, target);
+        unbufOut = saslStreams.out;
+        unbufIn = saslStreams.in;
+        out = new DataOutputStream(
+            new BufferedOutputStream(unbufOut, ioFileBufferSize));
+        in = new DataInputStream(
+            new BufferedInputStream(unbufIn, ioFileBufferSize));
+        sendRequest(out, block, accessToken, source, targetStorageType);
+        receiveResponse(in);
+
+        LOG.debug(
+            "Successfully moved block:{} from src:{} to destin:{} for"
+                + " satisfying storageType:{}",
+            block, source, target, targetStorageType);
+      } catch (IOException e) {
+        // TODO: handle failure retries
+        LOG.warn(
+            "Failed to move block:{} from src:{} to destin:{} to satisfy "
+                + "storageType:{}",
+            block, source, target, targetStorageType, e);
+      } finally {
+        IOUtils.closeStream(out);
+        IOUtils.closeStream(in);
+        IOUtils.closeSocket(sock);
+      }
+    }
+
+    /** Send a reportedBlock replace request to the output stream. */
+    private void sendRequest(DataOutputStream out, ExtendedBlock eb,
+        Token<BlockTokenIdentifier> accessToken, DatanodeInfo srcDn,
+        StorageType destinStorageType) throws IOException {
+      new Sender(out).replaceBlock(eb, destinStorageType, accessToken,
+          srcDn.getDatanodeUuid(), srcDn);
+    }
+
+    /** Receive a reportedBlock copy response from the input stream. */
+    private void receiveResponse(DataInputStream in) throws IOException {
+      BlockOpResponseProto response = BlockOpResponseProto
+          .parseFrom(vintPrefixed(in));
+      while (response.getStatus() == Status.IN_PROGRESS) {
+        // read intermediate responses
+        response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
+      }
+      String logInfo = "reportedBlock move is failed";
+      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c118eab/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
new file mode 100644
index 0000000..42ba265
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+
+/**
+ * A BlockStorageMovementCommand is an instruction to a DataNode to move the
+ * given set of blocks to specified target DataNodes to fulfill the block
+ * storage policy.
+ *
+ * Upon receiving this command, this DataNode coordinates all the block movement
+ * by passing the details to
+ * {@link org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker}
+ * service. After the block movement this DataNode sends response back to the
+ * NameNode about the movement status.
+ */
+public class BlockStorageMovementCommand extends DatanodeCommand {
+
+  // TODO: constructor needs to be refined based on the block movement data
+  // structure.
+  BlockStorageMovementCommand(int action) {
+    super(action);
+  }
+
+  /**
+   * Stores block to storage info that can be used for block movement.
+   */
+  public static class BlockMovingInfo {
+    private ExtendedBlock blk;
+    private DatanodeInfo[] sourceNodes;
+    private StorageType[] sourceStorageTypes;
+    private DatanodeInfo[] targetNodes;
+    private StorageType[] targetStorageTypes;
+
+    public BlockMovingInfo(ExtendedBlock block,
+        DatanodeInfo[] sourceDnInfos, DatanodeInfo[] targetDnInfos,
+        StorageType[] srcStorageTypes, StorageType[] targetStorageTypes) {
+      this.blk = block;
+      this.sourceNodes = sourceDnInfos;
+      this.targetNodes = targetDnInfos;
+      this.sourceStorageTypes = srcStorageTypes;
+      this.targetStorageTypes = targetStorageTypes;
+    }
+
+    public void addBlock(ExtendedBlock block) {
+      this.blk = block;
+    }
+
+    public ExtendedBlock getBlock() {
+      return this.blk;
+    }
+
+    public DatanodeInfo[] getSources() {
+      return sourceNodes;
+    }
+
+    public DatanodeInfo[] getTargets() {
+      return targetNodes;
+    }
+
+    public StorageType[] getTargetStorageTypes() {
+      return targetStorageTypes;
+    }
+
+    public StorageType[] getSourceStorageTypes() {
+      return sourceStorageTypes;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("BlockMovingInfo(\n  ")
+          .append("Moving block: ").append(blk).append(" From: ")
+          .append(Arrays.asList(sourceNodes)).append(" To: [")
+          .append(Arrays.asList(targetNodes)).append(")\n")
+          .append(" sourceStorageTypes: ")
+          .append(Arrays.toString(sourceStorageTypes))
+          .append(" targetStorageTypes: ")
+          .append(Arrays.toString(targetStorageTypes)).toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c118eab/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
new file mode 100644
index 0000000..c722306
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Supplier;
+
+/**
+ * This class tests the behavior of moving block replica to the given storage
+ * type to fulfill the storage policy requirement.
+ */
+public class TestStoragePolicySatisfyWorker {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestStoragePolicySatisfyWorker.class);
+
+  private static final int DEFAULT_BLOCK_SIZE = 100;
+
+  private static void initConf(Configuration conf) {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
+        1L);
+    conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
+  }
+
+  /**
+   * Tests to verify that the block replica is moving to ARCHIVE storage type to
+   * fulfill the storage policy requirement.
+   */
+  @Test(timeout = 120000)
+  public void testMoveSingleBlockToAnotherDatanode() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    initConf(conf);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(4)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE}})
+        .build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testMoveSingleBlockToAnotherDatanode";
+      // write to DISK
+      final FSDataOutputStream out = dfs.create(new Path(file), (short) 2);
+      out.writeChars("testMoveSingleBlockToAnotherDatanode");
+      out.close();
+
+      // verify before movement
+      LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+      StorageType[] storageTypes = lb.getStorageTypes();
+      for (StorageType storageType : storageTypes) {
+        Assert.assertTrue(StorageType.DISK == storageType);
+      }
+      // move to ARCHIVE
+      dfs.setStoragePolicy(new Path(file), "COLD");
+
+      lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+      DataNode src = cluster.getDataNodes().get(3);
+      DatanodeInfo targetDnInfo = DFSTestUtil
+          .getLocalDatanodeInfo(src.getXferPort());
+
+      // TODO: Need to revisit this when NN is implemented to be able to send
+      // block moving commands.
+      StoragePolicySatisfyWorker worker = new StoragePolicySatisfyWorker(conf,
+          src);
+      List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+      BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
+          lb.getBlock(), lb.getLocations()[0], targetDnInfo,
+          lb.getStorageTypes()[0], StorageType.ARCHIVE);
+      blockMovingInfos.add(blockMovingInfo);
+      INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
+      worker.processBlockMovingTasks(inode.getId(),
+          blockMovingInfos);
+      cluster.triggerHeartbeats();
+
+      // Wait till NameNode notified about the block location details
+      waitForLocatedBlockWithArchiveStorageType(dfs, file, 1, 30000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void waitForLocatedBlockWithArchiveStorageType(
+      final DistributedFileSystem dfs, final String file,
+      int expectedArchiveCount, int timeout) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LocatedBlock lb = null;
+        try {
+          lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+        } catch (IOException e) {
+          LOG.error("Exception while getting located blocks", e);
+          return false;
+        }
+
+        int archiveCount = 0;
+        for (StorageType storageType : lb.getStorageTypes()) {
+          if (StorageType.ARCHIVE == storageType) {
+            archiveCount++;
+          }
+        }
+        LOG.info("Archive replica count, expected={} and actual={}",
+            expectedArchiveCount, archiveCount);
+        return expectedArchiveCount == archiveCount;
+      }
+    }, 100, timeout);
+  }
+
+  BlockMovingInfo prepareBlockMovingInfo(ExtendedBlock block,
+      DatanodeInfo src, DatanodeInfo destin, StorageType storageType,
+      StorageType targetStorageType) {
+    return new BlockMovingInfo(block, new DatanodeInfo[] {src},
+        new DatanodeInfo[] {destin}, new StorageType[] {storageType},
+        new StorageType[] {targetStorageType});
+  }
+}


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


[05/50] [abbrv] hadoop git commit: HDFS-10801. [SPS]: Protocol buffer changes for sending storage movement commands from NN to DN. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-10801. [SPS]: Protocol buffer changes for sending storage movement commands from NN to DN. 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/4eaf873c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4eaf873c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4eaf873c

Branch: refs/heads/HDFS-10285
Commit: 4eaf873cf9667df25c9ed6e74e124cd53113d7d3
Parents: d0b52ab
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Tue Oct 11 11:44:06 2016 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:20:02 2017 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 90 ++++++++++++++++++++
 .../blockmanagement/DatanodeDescriptor.java     | 15 ----
 .../server/blockmanagement/DatanodeManager.java | 13 +++
 .../hdfs/server/datanode/BPOfferService.java    |  8 ++
 .../hadoop/hdfs/server/datanode/DataNode.java   |  8 +-
 .../datanode/StoragePolicySatisfyWorker.java    | 22 ++++-
 .../protocol/BlockStorageMovementCommand.java   | 71 ++++++++++++++-
 .../hdfs/server/protocol/DatanodeProtocol.java  |  1 +
 .../src/main/proto/DatanodeProtocol.proto       | 22 +++++
 .../namenode/TestStoragePolicySatisfier.java    | 86 +++++++++++--------
 10 files changed, 276 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 6539d32..83f3454 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -52,6 +52,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SlowPeerReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockStorageMovementCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockStorageMovementProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstructionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -94,6 +96,8 @@ import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.Block
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
@@ -465,6 +469,8 @@ public class PBHelper {
       return PBHelper.convert(proto.getBlkIdCmd());
     case BlockECReconstructionCommand:
       return PBHelper.convert(proto.getBlkECReconstructionCmd());
+    case BlockStorageMovementCommand:
+      return PBHelper.convert(proto.getBlkStorageMovementCmd());
     default:
       return null;
     }
@@ -599,6 +605,11 @@ public class PBHelper {
           .setBlkECReconstructionCmd(
               convert((BlockECReconstructionCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockStorageMovementCommand)
+          .setBlkStorageMovementCmd(
+              convert((BlockStorageMovementCommand) datanodeCommand));
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);
@@ -1096,4 +1107,83 @@ public class PBHelper {
         DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION,
         blkECReconstructionInfos);
   }
+
+  private static BlockStorageMovementCommandProto convert(
+      BlockStorageMovementCommand blkStorageMovementCmd) {
+    BlockStorageMovementCommandProto.Builder builder =
+        BlockStorageMovementCommandProto.newBuilder();
+
+    builder.setTrackID(blkStorageMovementCmd.getTrackID());
+    builder.setBlockPoolId(blkStorageMovementCmd.getBlockPoolId());
+    Collection<BlockMovingInfo> blockMovingInfos = blkStorageMovementCmd
+        .getBlockMovingTasks();
+    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
+      builder.addBlockStorageMovement(
+          convertBlockMovingInfo(blkMovingInfo));
+    }
+    return builder.build();
+  }
+
+  private static BlockStorageMovementProto convertBlockMovingInfo(
+      BlockMovingInfo blkMovingInfo) {
+    BlockStorageMovementProto.Builder builder = BlockStorageMovementProto
+        .newBuilder();
+    builder.setBlock(PBHelperClient.convert(blkMovingInfo.getBlock()));
+
+    DatanodeInfo[] sourceDnInfos = blkMovingInfo.getSources();
+    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
+
+    DatanodeInfo[] targetDnInfos = blkMovingInfo.getTargets();
+    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
+
+    StorageType[] sourceStorageTypes = blkMovingInfo.getSourceStorageTypes();
+    builder.setSourceStorageTypes(convertStorageTypesProto(sourceStorageTypes));
+
+    StorageType[] targetStorageTypes = blkMovingInfo.getTargetStorageTypes();
+    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
+
+    return builder.build();
+  }
+
+  private static DatanodeCommand convert(
+      BlockStorageMovementCommandProto blkStorageMovementCmdProto) {
+    Collection<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+    List<BlockStorageMovementProto> blkSPSatisfyList =
+        blkStorageMovementCmdProto.getBlockStorageMovementList();
+    for (BlockStorageMovementProto blkSPSatisfy : blkSPSatisfyList) {
+      blockMovingInfos.add(convertBlockMovingInfo(blkSPSatisfy));
+    }
+    return new BlockStorageMovementCommand(
+        DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT,
+        blkStorageMovementCmdProto.getTrackID(),
+        blkStorageMovementCmdProto.getBlockPoolId(), blockMovingInfos);
+  }
+
+  private static BlockMovingInfo convertBlockMovingInfo(
+      BlockStorageMovementProto blockStoragePolicySatisfyProto) {
+    BlockProto blockProto = blockStoragePolicySatisfyProto.getBlock();
+    Block block = PBHelperClient.convert(blockProto);
+
+    DatanodeInfosProto sourceDnInfosProto = blockStoragePolicySatisfyProto
+        .getSourceDnInfos();
+    DatanodeInfo[] sourceDnInfos = PBHelperClient.convert(sourceDnInfosProto);
+
+    DatanodeInfosProto targetDnInfosProto = blockStoragePolicySatisfyProto
+        .getTargetDnInfos();
+    DatanodeInfo[] targetDnInfos = PBHelperClient.convert(targetDnInfosProto);
+
+    StorageTypesProto srcStorageTypesProto = blockStoragePolicySatisfyProto
+        .getSourceStorageTypes();
+    StorageType[] srcStorageTypes = PBHelperClient.convertStorageTypes(
+        srcStorageTypesProto.getStorageTypesList(),
+        srcStorageTypesProto.getStorageTypesList().size());
+
+    StorageTypesProto targetStorageTypesProto = blockStoragePolicySatisfyProto
+        .getTargetStorageTypes();
+    StorageType[] targetStorageTypes = PBHelperClient.convertStorageTypes(
+        targetStorageTypesProto.getStorageTypesList(),
+        targetStorageTypesProto.getStorageTypesList().size());
+    return new BlockMovingInfo(block, sourceDnInfos, targetDnInfos,
+        srcStorageTypes, targetStorageTypes);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 46a45ee..1db002b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -1039,19 +1039,4 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public List<BlockMovingInfo> getBlocksToMoveStorages() {
     return storageMovementBlocks.poll();
   }
-
-  // TODO: we will remove this method once DN side handling integrated. We can
-  // convert the test to check real block movements instead of this ds.
-  @VisibleForTesting
-  public List<BlockMovingInfo> getStorageMovementPendingItems() {
-    List<BlockMovingInfo> flatList = new ArrayList<>();
-    Iterator<List<BlockMovingInfo>> iterator = storageMovementBlocks
-        .iterator();
-    while (iterator.hasNext()) {
-      List<BlockMovingInfo> next = iterator.next();
-      flatList.addAll(next);
-    }
-    return flatList;
-  }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 13c730d..6a5cfbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1737,6 +1737,19 @@ public class DatanodeManager {
       }
     }
 
+    // check pending block storage movement tasks
+    List<BlockMovingInfo> pendingBlockMovementList = nodeinfo
+        .getBlocksToMoveStorages();
+    if (pendingBlockMovementList != null) {
+      // TODO: trackID is used to track the block movement sends to coordinator
+      // datanode. Need to implement tracking logic. Temporarily, using a
+      // constant value -1.
+      long trackID = -1;
+      cmds.add(new BlockStorageMovementCommand(
+          DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT, trackID, blockPoolId,
+          pendingBlockMovementList));
+    }
+
     if (!cmds.isEmpty()) {
       return cmds.toArray(new DatanodeCommand[cmds.size()]);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index dbf7c8d..c77fe2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -781,6 +781,13 @@ class BPOfferService {
           ((BlockECReconstructionCommand) cmd).getECTasks();
       dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
       break;
+    case DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT:
+      LOG.info("DatanodeCommand action: DNA_BLOCK_STORAGE_MOVEMENT");
+      BlockStorageMovementCommand blkSPSCmd = (BlockStorageMovementCommand) cmd;
+      dn.getStoragePolicySatisfyWorker().processBlockMovingTasks(
+          blkSPSCmd.getTrackID(), blkSPSCmd.getBlockPoolId(),
+          blkSPSCmd.getBlockMovingTasks());
+      break;
     default:
       LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
     }
@@ -811,6 +818,7 @@ class BPOfferService {
     case DatanodeProtocol.DNA_CACHE:
     case DatanodeProtocol.DNA_UNCACHE:
     case DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION:
+    case DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       break;
     default:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 6163d93..9a8de99 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -384,6 +384,7 @@ public class DataNode extends ReconfigurableBase
   private String dnUserName = null;
   private BlockRecoveryWorker blockRecoveryWorker;
   private ErasureCodingWorker ecWorker;
+  private StoragePolicySatisfyWorker storagePolicySatisfyWorker;
   private final Tracer tracer;
   private final TracerConfigurationManager tracerConfigurationManager;
   private static final int NUM_CORES = Runtime.getRuntime()
@@ -1420,6 +1421,8 @@ public class DataNode extends ReconfigurableBase
 
     ecWorker = new ErasureCodingWorker(getConf(), this);
     blockRecoveryWorker = new BlockRecoveryWorker(this);
+    storagePolicySatisfyWorker =
+        new StoragePolicySatisfyWorker(getConf(), this);
 
     blockPoolManager = new BlockPoolManager(this);
     blockPoolManager.refreshNamenodes(getConf());
@@ -3620,4 +3623,7 @@ public class DataNode extends ReconfigurableBase
     }
     return volumeInfoList;
   }
-}
+
+  StoragePolicySatisfyWorker getStoragePolicySatisfyWorker() {
+    return storagePolicySatisfyWorker;
+  }}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index fa408f6..2c99963 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -28,6 +28,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.Socket;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -126,8 +127,25 @@ public class StoragePolicySatisfyWorker {
     return moverThreadPool;
   }
 
+  /**
+   * Handles the given set of block movement tasks. This will iterate over the
+   * block movement list and submit each block movement task asynchronously in a
+   * separate thread. Each task will move the block replica to the target node &
+   * wait for the completion.
+   *
+   * TODO: Presently this function is a blocking call, this has to be refined by
+   * moving the tracking logic to another tracker thread. HDFS-10884 jira
+   * addresses the same.
+   *
+   * @param trackID
+   *          unique tracking identifier
+   * @param blockPoolID
+   *          block pool ID
+   * @param blockMovingInfos
+   *          list of blocks to be moved
+   */
   public void processBlockMovingTasks(long trackID, String blockPoolID,
-      List<BlockMovingInfo> blockMovingInfos) {
+      Collection<BlockMovingInfo> blockMovingInfos) {
     Future<Void> moveCallable = null;
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       assert blkMovingInfo
@@ -143,8 +161,6 @@ public class StoragePolicySatisfyWorker {
       }
     }
 
-    // TODO: Presently this function act as a blocking call, this has to be
-    // refined by moving the tracking logic to another tracker thread.
     for (int i = 0; i < moverTaskFutures.size(); i++) {
       try {
         moveCallable = moverExecutorCompletionService.take();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
index c1ab800..7c97f1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.protocol;
 
 import java.util.Arrays;
+import java.util.Collection;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -33,12 +34,60 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
  * {@link org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker}
  * service. After the block movement this DataNode sends response back to the
  * NameNode about the movement status.
+ *
+ * The coordinator datanode will use 'trackId' identifier to coordinate the block
+ * movement of the given set of blocks. TrackId is a unique identifier that
+ * represents a group of blocks. Namenode will generate this unique value and
+ * send it to the coordinator datanode along with the
+ * BlockStorageMovementCommand. Datanode will monitor the completion of the
+ * block movements that grouped under this trackId and notifies Namenode about
+ * the completion status.
  */
 public class BlockStorageMovementCommand extends DatanodeCommand {
-  // TODO: constructor needs to be refined based on the block movement data
-  // structure.
-  BlockStorageMovementCommand(int action) {
+  private final long trackID;
+  private final String blockPoolId;
+  private final Collection<BlockMovingInfo> blockMovingTasks;
+
+  /**
+   * Block storage movement command constructor.
+   *
+   * @param action
+   *          protocol specific action
+   * @param trackID
+   *          unique identifier to monitor the given set of block movements
+   * @param blockPoolId
+   *          block pool ID
+   * @param blockMovingInfos
+   *          block to storage info that will be used for movement
+   */
+  public BlockStorageMovementCommand(int action, long trackID,
+      String blockPoolId, Collection<BlockMovingInfo> blockMovingInfos) {
     super(action);
+    this.trackID = trackID;
+    this.blockPoolId = blockPoolId;
+    this.blockMovingTasks = blockMovingInfos;
+  }
+
+  /**
+   * Returns trackID, which will be used to monitor the block movement assigned
+   * to this coordinator datanode.
+   */
+  public long getTrackID() {
+    return trackID;
+  }
+
+  /**
+   * Returns block pool ID.
+   */
+  public String getBlockPoolId() {
+    return blockPoolId;
+  }
+
+  /**
+   * Returns the list of blocks to be moved.
+   */
+  public Collection<BlockMovingInfo> getBlockMovingTasks() {
+    return blockMovingTasks;
   }
 
   /**
@@ -47,10 +96,24 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
   public static class BlockMovingInfo {
     private Block blk;
     private DatanodeInfo[] sourceNodes;
-    private StorageType[] sourceStorageTypes;
     private DatanodeInfo[] targetNodes;
+    private StorageType[] sourceStorageTypes;
     private StorageType[] targetStorageTypes;
 
+    /**
+     * Block to storage info constructor.
+     *
+     * @param block
+     *          block
+     * @param sourceDnInfos
+     *          node that can be the sources of a block move
+     * @param targetDnInfos
+     *          target datanode info
+     * @param srcStorageTypes
+     *          type of source storage media
+     * @param targetStorageTypes
+     *          type of destin storage media
+     */
     public BlockMovingInfo(Block block,
         DatanodeInfo[] sourceDnInfos, DatanodeInfo[] targetDnInfos,
         StorageType[] srcStorageTypes, StorageType[] targetStorageTypes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 1f55100..283f367 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -79,6 +79,7 @@ public interface DatanodeProtocol {
   final static int DNA_CACHE = 9;      // cache blocks
   final static int DNA_UNCACHE = 10;   // uncache blocks
   final static int DNA_ERASURE_CODING_RECONSTRUCTION = 11; // erasure coding reconstruction command
+  final static int DNA_BLOCK_STORAGE_MOVEMENT = 12; // block storage movement command
 
   /** 
    * Register Datanode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index bf0df5b..8e19809 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -60,6 +60,7 @@ message DatanodeCommandProto {
     NullDatanodeCommand = 7;
     BlockIdCommand = 8;
     BlockECReconstructionCommand = 9;
+    BlockStorageMovementCommand = 10;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -74,6 +75,7 @@ message DatanodeCommandProto {
   optional RegisterCommandProto registerCmd = 7;
   optional BlockIdCommandProto blkIdCmd = 8;
   optional BlockECReconstructionCommandProto blkECReconstructionCmd = 9;
+  optional BlockStorageMovementCommandProto blkStorageMovementCmd = 10;
 }
 
 /**
@@ -154,6 +156,26 @@ message BlockECReconstructionCommandProto {
   repeated BlockECReconstructionInfoProto blockECReconstructioninfo = 1;
 }
 
+ /**
+ * Block storage movement command
+ */
+message BlockStorageMovementCommandProto {
+  required uint64 trackID = 1;
+  required string blockPoolId = 2;
+  repeated BlockStorageMovementProto blockStorageMovement = 3;
+}
+
+/**
+ * Block storage movement information
+ */
+message BlockStorageMovementProto {
+  required BlockProto block = 1;
+  required DatanodeInfosProto sourceDnInfos = 2;
+  required DatanodeInfosProto targetDnInfos = 3;
+  required StorageTypesProto sourceStorageTypes = 4;
+  required StorageTypesProto targetStorageTypes = 5;
+}
+
 /**
  * registration - Information of the datanode registering with the namenode
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaf873c/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 b61814d..37664b5 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
@@ -18,9 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -29,8 +26,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Test;
@@ -74,9 +70,6 @@ public class TestStoragePolicySatisfier {
     try {
       // Change policy to ALL_SSD
       distributedFS.setStoragePolicy(new Path(file), "COLD");
-      Set<DatanodeDescriptor> previousNodes =
-          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
-              .getDatanodeManager().getDatanodes();
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -91,8 +84,8 @@ public class TestStoragePolicySatisfier {
 
       hdfsCluster.triggerHeartbeats();
       // Wait till namenode notified about the block location details
-      waitExpectedStorageType(StorageType.ARCHIVE, distributedFS, previousNodes,
-          6, 30000);
+      waitExpectedStorageType(file, StorageType.ARCHIVE, distributedFS, 3,
+          30000);
     } finally {
       hdfsCluster.shutdown();
     }
@@ -104,9 +97,6 @@ public class TestStoragePolicySatisfier {
     try {
       // Change policy to ALL_SSD
       distributedFS.setStoragePolicy(new Path(file), "ALL_SSD");
-      Set<DatanodeDescriptor> previousNodes =
-          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
-              .getDatanodeManager().getDatanodes();
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -123,8 +113,34 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      waitExpectedStorageType(StorageType.SSD, distributedFS, previousNodes, 6,
-          30000);
+      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 3, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToONESSD()
+      throws Exception {
+    try {
+      // Change policy to ONE_SSD
+      distributedFS.setStoragePolicy(new Path(file), "ONE_SSD");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, distributedFS, 2, 30000);
     } finally {
       hdfsCluster.shutdown();
     }
@@ -174,35 +190,31 @@ public class TestStoragePolicySatisfier {
     return cluster;
   }
 
-  // TODO: this assertion can be changed to end to end based assertion later
-  // when DN side processing work integrated to this work.
-  private void waitExpectedStorageType(final StorageType expectedStorageType,
-      final DistributedFileSystem dfs,
-      final Set<DatanodeDescriptor> previousNodes, int expectedArchiveCount,
-      int timeout) throws Exception {
+  // Check whether the Block movement has been successfully completed to satisfy
+  // the storage policy for the given file.
+  private void waitExpectedStorageType(final String fileName,
+      final StorageType expectedStorageType, final DistributedFileSystem dfs,
+      int expectedStorageCount, int timeout) throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        Iterator<DatanodeDescriptor> iterator = previousNodes.iterator();
-        int archiveCount = 0;
-        while (iterator.hasNext()) {
-          DatanodeDescriptor dn = iterator.next();
-          List<BlockMovingInfo> pendingItemsToMove =
-              dn.getStorageMovementPendingItems();
-          for (BlockMovingInfo blkInfoToMoveStorage : pendingItemsToMove) {
-            StorageType[] targetStorageTypes =
-                blkInfoToMoveStorage.getTargetStorageTypes();
-            for (StorageType storageType : targetStorageTypes) {
-              if (storageType == expectedStorageType) {
-                archiveCount++;
-              }
-            }
+        LocatedBlock lb = null;
+        try {
+          lb = dfs.getClient().getLocatedBlocks(fileName, 0).get(0);
+        } catch (IOException e) {
+          LOG.error("Exception while getting located blocks", e);
+          return false;
+        }
+        int actualStorageCount = 0;
+        for (StorageType storageType : lb.getStorageTypes()) {
+          if (expectedStorageType == storageType) {
+            actualStorageCount++;
           }
         }
         LOG.info(
             expectedStorageType + " replica count, expected={} and actual={}",
-            expectedArchiveCount, archiveCount);
-        return expectedArchiveCount == archiveCount;
+            expectedStorageType, actualStorageCount);
+        return expectedStorageCount == actualStorageCount;
       }
     }, 100, timeout);
   }


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


[19/50] [abbrv] hadoop git commit: HDFS-11150: [SPS]: Provide persistence when satisfying storage policy. Contributed by Yuanbo Liu

Posted by ra...@apache.org.
HDFS-11150: [SPS]: Provide persistence when satisfying storage policy. Contributed by Yuanbo Liu


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

Branch: refs/heads/HDFS-10285
Commit: 57f0b2ff138185a4e6380ca78c08dcbc1f6824b3
Parents: 8dcd716
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Wed Jan 11 13:48:58 2017 -0800
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:51 2017 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   2 +-
 .../hdfs/server/common/HdfsServerConstants.java |   3 +
 .../hdfs/server/namenode/FSDirAttrOp.java       |  81 +++--
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   8 +
 .../hdfs/server/namenode/FSDirectory.java       |  14 +
 .../hdfs/server/namenode/FSNamesystem.java      |   6 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  13 +-
 .../server/namenode/StoragePolicySatisfier.java |  22 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  37 +++
 .../TestPersistentStoragePolicySatisfier.java   | 311 +++++++++++++++++++
 .../namenode/TestStoragePolicySatisfier.java    | 112 +++----
 ...stStoragePolicySatisfierWithStripedFile.java |  17 +-
 12 files changed, 532 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index ae19e27..e72975d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1711,7 +1711,7 @@ public interface ClientProtocol {
    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException append not
    *           allowed in safemode.
    */
-  @Idempotent
+  @AtMostOnce
   void satisfyStoragePolicy(String path) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index e486317..42a2fc6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -365,6 +365,9 @@ public interface HdfsServerConstants {
   String XATTR_ERASURECODING_POLICY =
       "system.hdfs.erasurecoding.policy";
 
+  String XATTR_SATISFY_STORAGE_POLICY =
+      "system.hdfs.satisfy.storage.policy";
+
   Path MOVER_ID_PATH = new Path("/system/mover.id");
 
   long BLOCK_GROUP_INDEX_MASK = 15;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 7743f0e..991e855 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -42,12 +43,14 @@ import com.google.common.collect.Lists;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 
 public class FSDirAttrOp {
   static FileStatus setPermission(
@@ -197,10 +200,11 @@ public class FSDirAttrOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  static void satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
-      String src) throws IOException {
+  static FileStatus satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
+      String src, boolean logRetryCache) throws IOException {
 
     FSPermissionChecker pc = fsd.getPermissionChecker();
+    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -210,10 +214,13 @@ public class FSDirAttrOp {
       if (fsd.isPermissionEnabled()) {
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
       }
-      unprotectedSatisfyStoragePolicy(bm, iip);
+      XAttr satisfyXAttr = unprotectedSatisfyStoragePolicy(iip, bm, fsd);
+      xAttrs.add(satisfyXAttr);
     } finally {
       fsd.writeUnlock();
     }
+    fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+    return fsd.getAuditFileInfo(iip);
   }
 
   static BlockStoragePolicy[] getStoragePolicies(BlockManager bm)
@@ -477,33 +484,61 @@ public class FSDirAttrOp {
     }
   }
 
-  static void unprotectedSatisfyStoragePolicy(BlockManager bm,
-      INodesInPath iip) throws IOException {
+  static XAttr unprotectedSatisfyStoragePolicy(INodesInPath iip,
+      BlockManager bm, FSDirectory fsd) throws IOException {
 
-    // check whether file exists.
-    INode inode = iip.getLastINode();
-    if (inode == null) {
-      throw new FileNotFoundException("File/Directory does not exist: "
-          + iip.getPath());
-    }
+    final INode inode = FSDirectory.resolveLastINode(iip);
+    final int snapshotId = iip.getLatestSnapshotId();
+    final List<INode> candidateNodes = new ArrayList<>();
 
-    // TODO: need to check whether inode's storage policy
-    // has been satisfied or inode exists in the satisfier
-    // list before calling satisfyStoragePolicy in BlockManager.
-    if (inode.isDirectory()) {
-      final int snapshotId = iip.getLatestSnapshotId();
+    // TODO: think about optimization here, label the dir instead
+    // of the sub-files of the dir.
+    if (inode.isFile()) {
+      candidateNodes.add(inode);
+    } else if (inode.isDirectory()) {
       for (INode node : inode.asDirectory().getChildrenList(snapshotId)) {
         if (node.isFile()) {
-          bm.satisfyStoragePolicy(node.getId());
-
+          candidateNodes.add(node);
         }
       }
-    } else if (inode.isFile()) {
-      bm.satisfyStoragePolicy(inode.getId());
-    } else {
-      throw new FileNotFoundException("File/Directory does not exist: "
-          + iip.getPath());
     }
+
+    // If node has satisfy xattr, then stop adding it
+    // to satisfy movement queue.
+    if (inodeHasSatisfyXAttr(candidateNodes)) {
+      throw new IOException(
+          "Cannot request to call satisfy storage policy on path "
+          + iip.getPath()
+          + ", as this file/dir was already called for satisfying "
+          + "storage policy.");
+    }
+
+    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+    final XAttr satisfyXAttr =
+        XAttrHelper.buildXAttr(XATTR_SATISFY_STORAGE_POLICY);
+    xattrs.add(satisfyXAttr);
+
+    for (INode node : candidateNodes) {
+      bm.satisfyStoragePolicy(node.getId());
+      List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(node);
+      List<XAttr> newXAttrs = FSDirXAttrOp.setINodeXAttrs(
+          fsd, existingXAttrs, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
+      XAttrStorage.updateINodeXAttrs(node, newXAttrs, snapshotId);
+    }
+    return satisfyXAttr;
+  }
+
+  private static boolean inodeHasSatisfyXAttr(List<INode> candidateNodes) {
+    // If the node is a directory and one of the child files
+    // has satisfy xattr, then return true for this directory.
+    for (INode inode : candidateNodes) {
+      final XAttrFeature f = inode.getXAttrFeature();
+      if (inode.isFile() &&
+          f != null && f.getXAttr(XATTR_SATISFY_STORAGE_POLICY) != null) {
+        return true;
+      }
+    }
+    return false;
   }
 
   private static void setDirStoragePolicy(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
index 3223467..b0bda10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
@@ -42,6 +42,7 @@ import java.util.ListIterator;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 
 class FSDirXAttrOp {
   private static final XAttr KEYID_XATTR =
@@ -288,6 +289,13 @@ class FSDirXAttrOp {
         }
       }
 
+      // Add inode id to movement queue if xattrs contain satisfy xattr.
+      if (XATTR_SATISFY_STORAGE_POLICY.equals(xaName)) {
+        FSDirAttrOp.unprotectedSatisfyStoragePolicy(iip,
+            fsd.getBlockManager(), fsd);
+        continue;
+      }
+
       if (!isFile && SECURITY_XATTR_UNREADABLE_BY_SUPERUSER.equals(xaName)) {
         throw new IOException("Can only set '" +
             SECURITY_XATTR_UNREADABLE_BY_SUPERUSER + "' on a file.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index d119457..37ed027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -91,6 +91,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 
 /**
@@ -1398,10 +1399,23 @@ public class FSDirectory implements Closeable {
       if (!inode.isSymlink()) {
         final XAttrFeature xaf = inode.getXAttrFeature();
         addEncryptionZone((INodeWithAdditionalFields) inode, xaf);
+        addStoragePolicySatisfier((INodeWithAdditionalFields) inode, xaf);
       }
     }
   }
 
+  private void addStoragePolicySatisfier(INodeWithAdditionalFields inode,
+      XAttrFeature xaf) {
+    if (xaf == null || inode.isDirectory()) {
+      return;
+    }
+    XAttr xattr = xaf.getXAttr(XATTR_SATISFY_STORAGE_POLICY);
+    if (xattr == null) {
+      return;
+    }
+    getBlockManager().satisfyStoragePolicy(inode.getId());
+  }
+
   private void addEncryptionZone(INodeWithAdditionalFields inode,
       XAttrFeature xaf) {
     if (xaf == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/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 6d4e65f..ae38685 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
@@ -2162,7 +2162,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *
    * @param src file/directory path
    */
-  void satisfyStoragePolicy(String src) throws IOException {
+  void satisfyStoragePolicy(String src, boolean logRetryCache)
+      throws IOException {
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -2184,8 +2185,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 + " by admin. Seek for an admin help to activate it "
                 + "or use Mover tool.");
       }
-      // TODO: need to update editlog for persistence.
-      FSDirAttrOp.satisfyStoragePolicy(dir, blockManager, src);
+      FSDirAttrOp.satisfyStoragePolicy(dir, blockManager, src, logRetryCache);
     } finally {
       writeUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 5c49e1c..96c0fdf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1371,7 +1371,18 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public void satisfyStoragePolicy(String src) throws IOException {
     checkNNStartup();
-    namesystem.satisfyStoragePolicy(src);
+    namesystem.checkOperation(OperationCategory.WRITE);
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return; // Return previous response
+    }
+    boolean success = false;
+    try {
+      namesystem.satisfyStoragePolicy(src, cacheEntry != null);
+      success = true;
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
   }
 
   @Override // ClientProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/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 b1b1464..3b19833 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
@@ -168,16 +168,18 @@ public class StoragePolicySatisfier implements Runnable {
     }
     while (namesystem.isRunning() && isRunning) {
       try {
-        Long blockCollectionID = storageMovementNeeded.get();
-        if (blockCollectionID != null) {
-          BlockCollection blockCollection =
-              namesystem.getBlockCollection(blockCollectionID);
-          // Check blockCollectionId existence.
-          if (blockCollection != null) {
-            boolean allBlockLocsAttemptedToSatisfy =
-                computeAndAssignStorageMismatchedBlocksToDNs(blockCollection);
-            this.storageMovementsMonitor.add(blockCollectionID,
-                allBlockLocsAttemptedToSatisfy);
+        if (!namesystem.isInSafeMode()) {
+          Long blockCollectionID = storageMovementNeeded.get();
+          if (blockCollectionID != null) {
+            BlockCollection blockCollection =
+                namesystem.getBlockCollection(blockCollectionID);
+            // Check blockCollectionId existence.
+            if (blockCollection != null) {
+              boolean allBlockLocsAttemptedToSatisfy =
+                  computeAndAssignStorageMismatchedBlocksToDNs(blockCollection);
+              this.storageMovementsMonitor
+                  .add(blockCollectionID, allBlockLocsAttemptedToSatisfy);
+            }
           }
         }
         // TODO: We can think to make this as configurable later, how frequently

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 1411a7f..d04b8e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -2354,4 +2354,41 @@ public class DFSTestUtil {
     }
     return closedFiles;
   }
+
+  /**
+   * Check whether the Block movement has been successfully
+   * completed to satisfy the storage policy for the given file.
+   * @param fileName file name.
+   * @param expectedStorageType storage type.
+   * @param expectedStorageCount expected storage type.
+   * @param timeout timeout.
+   * @param fs distributedFileSystem.
+   * @throws Exception
+   */
+  public static void waitExpectedStorageType(String fileName,
+      final StorageType expectedStorageType, int expectedStorageCount,
+      int timeout, DistributedFileSystem fs) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        final LocatedBlock lb;
+        try {
+          lb = fs.getClient().getLocatedBlocks(fileName, 0).get(0);
+        } catch (IOException e) {
+          LOG.error("Exception while getting located blocks", e);
+          return false;
+        }
+        int actualStorageCount = 0;
+        for(StorageType type : lb.getStorageTypes()) {
+          if (expectedStorageType == type) {
+            actualStorageCount++;
+          }
+        }
+        LOG.info(
+            expectedStorageType + " replica count, expected="
+                + expectedStorageCount + " and actual=" + actualStorageCount);
+        return expectedStorageCount == actualStorageCount;
+      }
+    }, 1000, timeout);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/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
new file mode 100644
index 0000000..e4b4290
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPersistentStoragePolicySatisfier.java
@@ -0,0 +1,311 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Test persistence of satisfying files/directories.
+ */
+public class TestPersistentStoragePolicySatisfier {
+
+  private static Configuration conf;
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem fs;
+
+  private static Path testFile =
+      new Path("/testFile");
+  private static String testFileName = testFile.toString();
+
+  private static Path parentDir = new Path("/parentDir");
+  private static Path parentFile = new Path(parentDir, "parentFile");
+  private static String parentFileName = parentFile.toString();
+  private static Path childDir = new Path(parentDir, "childDir");
+  private static Path childFile = new Path(childDir, "childFile");
+  private static String childFileName = childFile.toString();
+
+  private static final String COLD = "COLD";
+  private static final String WARM = "WARM";
+  private static final String ONE_SSD = "ONE_SSD";
+  private static final String ALL_SSD = "ALL_SSD";
+
+  private static StorageType[][] storageTypes = new StorageType[][] {
+      {StorageType.ARCHIVE, StorageType.DISK},
+      {StorageType.DISK, StorageType.SSD},
+      {StorageType.SSD, StorageType.RAM_DISK},
+      {StorageType.ARCHIVE, StorageType.DISK},
+      {StorageType.ARCHIVE, StorageType.SSD}
+  };
+
+  private final int timeout = 300000;
+
+  /**
+   * Setup environment for every test case.
+   * @throws IOException
+   */
+  public void clusterSetUp() throws Exception {
+    clusterSetUp(false);
+  }
+
+  /**
+   * Setup cluster environment.
+   * @param isHAEnabled if true, enable simple HA.
+   * @throws IOException
+   */
+  private void clusterSetUp(boolean isHAEnabled) throws Exception {
+    conf = new HdfsConfiguration();
+    final int dnNumber = storageTypes.length;
+    final short replication = 3;
+    MiniDFSCluster.Builder clusterBuilder = new MiniDFSCluster.Builder(conf)
+        .storageTypes(storageTypes)
+        .numDataNodes(dnNumber);
+    if (isHAEnabled) {
+      clusterBuilder.nnTopology(MiniDFSNNTopology.simpleHATopology());
+    }
+    cluster = clusterBuilder.build();
+    cluster.waitActive();
+    if (isHAEnabled) {
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+    } else {
+      fs = cluster.getFileSystem();
+    }
+
+    createTestFiles(fs, replication);
+  }
+
+  /**
+   * Setup test files for testing.
+   * @param dfs
+   * @param replication
+   * @throws Exception
+   */
+  private void createTestFiles(DistributedFileSystem dfs,
+      short replication) throws Exception {
+    DFSTestUtil.createFile(dfs, testFile, 1024L, replication, 0L);
+    DFSTestUtil.createFile(dfs, parentFile, 1024L, replication, 0L);
+    DFSTestUtil.createFile(dfs, childFile, 1024L, replication, 0L);
+
+    DFSTestUtil.waitReplication(dfs, testFile, replication);
+    DFSTestUtil.waitReplication(dfs, parentFile, replication);
+    DFSTestUtil.waitReplication(dfs, childFile, replication);
+  }
+
+  /**
+   * Tear down environment for every test case.
+   * @throws IOException
+   */
+  private void clusterShutdown() throws IOException{
+    if(fs != null) {
+      fs.close();
+      fs = null;
+    }
+    if(cluster != null) {
+      cluster.shutdown(true);
+      cluster = null;
+    }
+  }
+
+  /**
+   * While satisfying file/directory, trigger the cluster's checkpoint to
+   * make sure satisfier persistence work as expected. This test case runs
+   * as below:
+   * 1. use satisfyStoragePolicy and add xAttr to the file.
+   * 2. do the checkpoint by secondary NameNode.
+   * 3. restart the cluster immediately.
+   * 4. make sure all the storage policies are satisfied.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testWithCheckpoint() throws Exception {
+    try {
+      clusterSetUp();
+      fs.setStoragePolicy(testFile, WARM);
+      fs.satisfyStoragePolicy(testFile);
+
+      // Start the checkpoint.
+      conf.set(
+          DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+      SecondaryNameNode secondary = new SecondaryNameNode(conf);
+      secondary.doCheckpoint();
+      restartCluster();
+
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.DISK, 1, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.ARCHIVE, 2, timeout, fs);
+
+      fs.setStoragePolicy(parentDir, COLD);
+      fs.satisfyStoragePolicy(parentDir);
+
+      DFSTestUtil.waitExpectedStorageType(
+          parentFileName, StorageType.ARCHIVE, 3, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          childFileName, StorageType.DEFAULT, 3, timeout, fs);
+
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
+   * Tests to verify satisfier persistence working as expected
+   * in HA env. This test case runs as below:
+   * 1. setup HA cluster env with simple HA topology.
+   * 2. switch the active NameNode from nn0/nn1 to nn1/nn0.
+   * 3. make sure all the storage policies are satisfied.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testWithHA() throws Exception {
+    try {
+      // Enable HA env for testing.
+      clusterSetUp(true);
+
+      fs.setStoragePolicy(testFile, ALL_SSD);
+      fs.satisfyStoragePolicy(testFile);
+
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.SSD, 3, timeout, fs);
+
+      // test directory
+      fs.setStoragePolicy(parentDir, WARM);
+      fs.satisfyStoragePolicy(parentDir);
+      cluster.transitionToStandby(1);
+      cluster.transitionToActive(0);
+
+      DFSTestUtil.waitExpectedStorageType(
+          parentFileName, StorageType.DISK, 1, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          parentFileName, StorageType.ARCHIVE, 2, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          childFileName, StorageType.DEFAULT, 3, timeout, fs);
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+
+  /**
+   * Tests to verify satisfier persistence working well with multiple
+   * restarts operations. This test case runs as below:
+   * 1. satisfy the storage policy of file1.
+   * 2. restart the cluster.
+   * 3. check whether all the blocks are satisfied.
+   * 4. satisfy the storage policy of file2.
+   * 5. restart the cluster.
+   * 6. check whether all the blocks are satisfied.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testWithRestarts() throws Exception {
+    try {
+      clusterSetUp();
+      fs.setStoragePolicy(testFile, ONE_SSD);
+      fs.satisfyStoragePolicy(testFile);
+      restartCluster();
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.SSD, 1, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.DISK, 2, timeout, fs);
+
+      // test directory
+      fs.setStoragePolicy(parentDir, COLD);
+      fs.satisfyStoragePolicy(parentDir);
+      restartCluster();
+      DFSTestUtil.waitExpectedStorageType(
+          parentFileName, StorageType.ARCHIVE, 3, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          childFileName, StorageType.DEFAULT, 3, timeout, fs);
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
+   * Tests to verify satisfier persistence working well with
+   * federal HA env. This test case runs as below:
+   * 1. setup HA test environment with federal topology.
+   * 2. satisfy storage policy of file1.
+   * 3. switch active NameNode from nn0 to nn1.
+   * 4. switch active NameNode from nn2 to nn3.
+   * 5. check whether the storage policy of file1 is satisfied.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testWithFederationHA() throws Exception {
+    try {
+      conf = new HdfsConfiguration();
+      final MiniDFSCluster haCluster = new MiniDFSCluster
+          .Builder(conf)
+          .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
+          .storageTypes(storageTypes)
+          .numDataNodes(storageTypes.length).build();
+      haCluster.waitActive();
+      haCluster.transitionToActive(1);
+      haCluster.transitionToActive(3);
+
+      fs = HATestUtil.configureFailoverFs(haCluster, conf);
+      createTestFiles(fs, (short) 3);
+
+      fs.setStoragePolicy(testFile, WARM);
+      fs.satisfyStoragePolicy(testFile);
+
+      haCluster.transitionToStandby(1);
+      haCluster.transitionToActive(0);
+      haCluster.transitionToStandby(3);
+      haCluster.transitionToActive(2);
+
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.DISK, 1, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          testFileName, StorageType.ARCHIVE, 2, timeout, fs);
+
+    } finally {
+      clusterShutdown();
+    }
+  }
+
+  /**
+   * Restart the hole env and trigger the DataNode's heart beats.
+   * @throws Exception
+   */
+  private void restartCluster() throws Exception {
+    cluster.restartDataNodes();
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    cluster.triggerHeartbeats();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/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 9abb78d..1c53894 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
@@ -108,7 +108,8 @@ public class TestStoragePolicySatisfier {
 
       hdfsCluster.triggerHeartbeats();
       // Wait till namenode notified about the block location details
-      waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 3, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -137,7 +138,8 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      waitExpectedStorageType(file, StorageType.SSD, 3, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 3, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -164,8 +166,10 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -195,8 +199,10 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
 
       // Wait till the block is moved to SSD areas
-      waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
 
       waitForBlocksMovementResult(1, 30000);
     } finally {
@@ -245,8 +251,10 @@ public class TestStoragePolicySatisfier {
 
       for (String fileName : files) {
         // Wait till the block is moved to SSD areas
-        waitExpectedStorageType(fileName, StorageType.SSD, 1, 30000);
-        waitExpectedStorageType(fileName, StorageType.DISK, 2, 30000);
+        DFSTestUtil.waitExpectedStorageType(
+            fileName, StorageType.SSD, 1, 30000, dfs);
+        DFSTestUtil.waitExpectedStorageType(
+            fileName, StorageType.DISK, 2, 30000, dfs);
       }
 
       waitForBlocksMovementResult(blockCollectionIds.size(), 30000);
@@ -279,7 +287,8 @@ public class TestStoragePolicySatisfier {
 
       hdfsCluster.triggerHeartbeats();
       // Wait till namenode notified about the block location details
-      waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 3, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -317,11 +326,14 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
 
       // take effect for the file in the directory.
-      waitExpectedStorageType(subFile1, StorageType.SSD, 1, 30000);
-      waitExpectedStorageType(subFile1, StorageType.DISK, 2, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          subFile1, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          subFile1, StorageType.DISK, 2, 30000, dfs);
 
       // take no effect for the sub-dir's file in the directory.
-      waitExpectedStorageType(subFile2, StorageType.DEFAULT, 3, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          subFile2, StorageType.DEFAULT, 3, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -367,6 +379,20 @@ public class TestStoragePolicySatisfier {
       } catch (FileNotFoundException e) {
 
       }
+
+      try {
+        hdfsAdmin.satisfyStoragePolicy(new Path(file));
+        hdfsAdmin.satisfyStoragePolicy(new Path(file));
+        Assert.fail(String.format(
+            "Should failed to satisfy storage policy "
+            + "for %s ,since it has been "
+            + "added to satisfy movement queue.", file));
+      } catch (IOException e) {
+        GenericTestUtils.assertExceptionContains(
+            String.format("Cannot request to call satisfy storage policy "
+                + "on path %s, as this file/dir was already called for "
+                + "satisfying storage policy.", file), e);
+      }
     } finally {
       shutdownCluster();
     }
@@ -407,8 +433,10 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier identified that block to move to
       // ARCHIVE area.
-      waitExpectedStorageType(file, StorageType.ARCHIVE, 1, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
 
       waitForBlocksMovementResult(1, 30000);
     } finally {
@@ -451,7 +479,8 @@ public class TestStoragePolicySatisfier {
       // No block movement will be scheduled as there is no target node available
       // with the required storage type.
       waitForAttemptedItems(1, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, 3, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 3, 30000, dfs);
       // Since there is no target node the item will get timed out and then
       // re-attempted.
       waitForAttemptedItems(1, 30000);
@@ -523,8 +552,10 @@ public class TestStoragePolicySatisfier {
     // with the required storage type.
     waitForAttemptedItems(1, 30000);
     waitForBlocksMovementResult(1, 30000);
-    waitExpectedStorageType(file1, StorageType.ARCHIVE, 1, 30000);
-    waitExpectedStorageType(file1, StorageType.DISK, 2, 30000);
+    DFSTestUtil.waitExpectedStorageType(
+        file1, StorageType.ARCHIVE, 1, 30000, dfs);
+    DFSTestUtil.waitExpectedStorageType(
+        file1, StorageType.DISK, 2, 30000, dfs);
   }
 
   /**
@@ -571,8 +602,10 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier identified that block to move to
       // ARCHIVE area.
-      waitExpectedStorageType(file, StorageType.ARCHIVE, 2, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, 3, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 2, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 3, 30000, dfs);
 
       waitForBlocksMovementResult(1, 30000);
     } finally {
@@ -606,8 +639,10 @@ public class TestStoragePolicySatisfier {
 
       namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
       hdfsCluster.triggerHeartbeats();
-      waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.SSD, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 2, 30000, dfs);
 
     } finally {
       shutdownCluster();
@@ -644,8 +679,10 @@ public class TestStoragePolicySatisfier {
       namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
       hdfsCluster.triggerHeartbeats();
 
-      waitExpectedStorageType(file, StorageType.DISK, 1, 30000);
-      waitExpectedStorageType(file, StorageType.ARCHIVE, 2, 30000);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.DISK, 1, 30000, dfs);
+      DFSTestUtil.waitExpectedStorageType(
+          file, StorageType.ARCHIVE, 2, 30000, dfs);
     } finally {
       shutdownCluster();
     }
@@ -771,33 +808,4 @@ public class TestStoragePolicySatisfier {
     cluster.waitActive();
     return cluster;
   }
-
-  // Check whether the Block movement has been successfully completed to satisfy
-  // the storage policy for the given file.
-  private void waitExpectedStorageType(final String fileName,
-      final StorageType expectedStorageType, int expectedStorageCount,
-      int timeout) throws Exception {
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        LocatedBlock lb = null;
-        try {
-          lb = dfs.getClient().getLocatedBlocks(fileName, 0).get(0);
-        } catch (IOException e) {
-          LOG.error("Exception while getting located blocks", e);
-          return false;
-        }
-        int actualStorageCount = 0;
-        for (StorageType storageType : lb.getStorageTypes()) {
-          if (expectedStorageType == storageType) {
-            actualStorageCount++;
-          }
-        }
-        LOG.info(
-            expectedStorageType + " replica count, expected={} and actual={}",
-            expectedStorageType, actualStorageCount);
-        return expectedStorageCount == actualStorageCount;
-      }
-    }, 100, timeout);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f0b2ff/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 5f8639f..eb4a6a3 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
@@ -66,7 +66,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
   private int defaultStripeBlockSize;
 
   private ErasureCodingPolicy getEcPolicy() {
-    return ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    return StripedFileTestUtil.getDefaultECPolicy();
   }
 
   /**
@@ -99,6 +99,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }
 
     final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        StripedFileTestUtil.getDefaultECPolicy().getName());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -128,7 +130,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
       client.mkdirs(barDir, new FsPermission((short) 777), true);
       client.setStoragePolicy(barDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
       // set an EC policy on "/bar" directory
-      client.setErasureCodingPolicy(barDir, null);
+      client.setErasureCodingPolicy(barDir,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
 
       // write file to barDir
       final String fooFile = "/bar/foo";
@@ -206,6 +209,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }
 
     final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        StripedFileTestUtil.getDefaultECPolicy().getName());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -235,7 +240,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
       client.mkdirs(barDir, new FsPermission((short) 777), true);
       client.setStoragePolicy(barDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
       // set an EC policy on "/bar" directory
-      client.setErasureCodingPolicy(barDir, null);
+      client.setErasureCodingPolicy(barDir,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
 
       // write file to barDir
       final String fooFile = "/bar/foo";
@@ -314,6 +320,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }
 
     final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        StripedFileTestUtil.getDefaultECPolicy().getName());
     initConfWithStripe(conf, defaultStripeBlockSize);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numOfDatanodes)
@@ -343,7 +351,8 @@ public class TestStoragePolicySatisfierWithStripedFile {
       client.mkdirs(barDir, new FsPermission((short) 777), true);
       client.setStoragePolicy(barDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
       // set an EC policy on "/bar" directory
-      client.setErasureCodingPolicy(barDir, null);
+      client.setErasureCodingPolicy(barDir,
+          StripedFileTestUtil.getDefaultECPolicy().getName());
 
       // write file to barDir
       final String fooFile = "/bar/foo";


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


[21/50] [abbrv] hadoop git commit: HDFS-11309. [SPS]: chooseTargetTypeInSameNode should pass accurate block size to chooseStorage4Block while choosing target. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11309. [SPS]: chooseTargetTypeInSameNode should pass accurate block size to chooseStorage4Block while choosing target. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: e6e6a59b994ea152465812941591010bcbe4cf1c
Parents: 66bf0e6
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Fri Jan 20 21:37:51 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:29:02 2017 +0530

----------------------------------------------------------------------
 .../server/namenode/StoragePolicySatisfier.java |  31 +++---
 .../namenode/TestStoragePolicySatisfier.java    | 108 ++++++++++++++++---
 2 files changed, 110 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6e6a59b/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 3b19833..1c48910 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
@@ -403,24 +403,25 @@ public class StoragePolicySatisfier implements Runnable {
     List<StorageType> sourceStorageTypes = new ArrayList<>();
     List<DatanodeInfo> targetNodes = new ArrayList<>();
     List<StorageType> targetStorageTypes = new ArrayList<>();
-    List<DatanodeDescriptor> chosenNodes = new ArrayList<>();
+    List<DatanodeDescriptor> excludeNodes = new ArrayList<>();
 
     // Looping over all the source node locations and choose the target
     // storage within same node if possible. This is done separately to
     // avoid choosing a target which already has this block.
     for (int i = 0; i < sourceWithStorageList.size(); i++) {
       StorageTypeNodePair existingTypeNodePair = sourceWithStorageList.get(i);
-      StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(
+      StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(blockInfo,
           existingTypeNodePair.dn, expected);
       if (chosenTarget != null) {
         sourceNodes.add(existingTypeNodePair.dn);
         sourceStorageTypes.add(existingTypeNodePair.storageType);
         targetNodes.add(chosenTarget.dn);
         targetStorageTypes.add(chosenTarget.storageType);
-        chosenNodes.add(chosenTarget.dn);
         expected.remove(chosenTarget.storageType);
         // TODO: We can increment scheduled block count for this node?
       }
+      // To avoid choosing this excludeNodes as targets later
+      excludeNodes.add(existingTypeNodePair.dn);
     }
 
     // Looping over all the source node locations. Choose a remote target
@@ -437,28 +438,28 @@ public class StoragePolicySatisfier implements Runnable {
           .getNetworkTopology().isNodeGroupAware()) {
         chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
             expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
-            chosenNodes);
+            excludeNodes);
       }
 
       // Then, match nodes on the same rack
       if (chosenTarget == null) {
         chosenTarget =
             chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
-                Matcher.SAME_RACK, locsForExpectedStorageTypes, chosenNodes);
+                Matcher.SAME_RACK, locsForExpectedStorageTypes, excludeNodes);
       }
 
       if (chosenTarget == null) {
         chosenTarget =
             chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
-                Matcher.ANY_OTHER, locsForExpectedStorageTypes, chosenNodes);
+                Matcher.ANY_OTHER, locsForExpectedStorageTypes, excludeNodes);
       }
       if (null != chosenTarget) {
         sourceNodes.add(existingTypeNodePair.dn);
         sourceStorageTypes.add(existingTypeNodePair.storageType);
         targetNodes.add(chosenTarget.dn);
         targetStorageTypes.add(chosenTarget.storageType);
-        chosenNodes.add(chosenTarget.dn);
         expected.remove(chosenTarget.storageType);
+        excludeNodes.add(chosenTarget.dn);
         // TODO: We can increment scheduled block count for this node?
       } else {
         LOG.warn(
@@ -554,14 +555,18 @@ public class StoragePolicySatisfier implements Runnable {
   /**
    * Choose the target storage within same datanode if possible.
    *
-   * @param source source datanode
-   * @param targetTypes list of target storage types
+   * @param block
+   *          - block info
+   * @param source
+   *          - source datanode
+   * @param targetTypes
+   *          - list of target storage types
    */
-  private StorageTypeNodePair chooseTargetTypeInSameNode(
+  private StorageTypeNodePair chooseTargetTypeInSameNode(Block block,
       DatanodeDescriptor source, List<StorageType> targetTypes) {
     for (StorageType t : targetTypes) {
       DatanodeStorageInfo chooseStorage4Block =
-          source.chooseStorage4Block(t, 0);
+          source.chooseStorage4Block(t, block.getNumBytes());
       if (chooseStorage4Block != null) {
         return new StorageTypeNodePair(t, source);
       }
@@ -572,7 +577,7 @@ public class StoragePolicySatisfier implements Runnable {
   private StorageTypeNodePair chooseTarget(Block block,
       DatanodeDescriptor source, List<StorageType> targetTypes, Matcher matcher,
       StorageTypeNodeMap locsForExpectedStorageTypes,
-      List<DatanodeDescriptor> chosenNodes) {
+      List<DatanodeDescriptor> excludeNodes) {
     for (StorageType t : targetTypes) {
       List<DatanodeDescriptor> nodesWithStorages =
           locsForExpectedStorageTypes.getNodesWithStorages(t);
@@ -581,7 +586,7 @@ public class StoragePolicySatisfier implements Runnable {
       }
       Collections.shuffle(nodesWithStorages);
       for (DatanodeDescriptor target : nodesWithStorages) {
-        if (!chosenNodes.contains(target) && matcher.match(
+        if (!excludeNodes.contains(target) && matcher.match(
             blockManager.getDatanodeManager().getNetworkTopology(), source,
             target)) {
           if (null != target.chooseStorage4Block(t, block.getNumBytes())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6e6a59b/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 1c53894..de73e8b 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
@@ -18,12 +18,14 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
+import static org.junit.Assert.assertNull;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -41,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
@@ -57,6 +60,8 @@ import com.google.common.base.Supplier;
  * moved and finding its suggested target locations to move.
  */
 public class TestStoragePolicySatisfier {
+  private static final String ONE_SSD = "ONE_SSD";
+  private static final String COLD = "COLD";
   private static final Logger LOG =
       LoggerFactory.getLogger(TestStoragePolicySatisfier.class);
   private final Configuration config = new HdfsConfiguration();
@@ -93,7 +98,7 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), "COLD");
+      dfs.setStoragePolicy(new Path(file), COLD);
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -151,7 +156,7 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), "ONE_SSD");
+      dfs.setStoragePolicy(new Path(file), ONE_SSD);
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -184,7 +189,7 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), "ONE_SSD");
+      dfs.setStoragePolicy(new Path(file), ONE_SSD);
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -232,7 +237,7 @@ public class TestStoragePolicySatisfier {
       List<Long> blockCollectionIds = new ArrayList<>();
       // Change policy to ONE_SSD
       for (String fileName : files) {
-        dfs.setStoragePolicy(new Path(fileName), "ONE_SSD");
+        dfs.setStoragePolicy(new Path(fileName), ONE_SSD);
         INode inode = namesystem.getFSDirectory().getINode(fileName);
         blockCollectionIds.add(inode.getId());
       }
@@ -274,12 +279,12 @@ public class TestStoragePolicySatisfier {
       HdfsAdmin hdfsAdmin =
           new HdfsAdmin(FileSystem.getDefaultUri(config), config);
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), "COLD");
+      dfs.setStoragePolicy(new Path(file), COLD);
 
       StorageType[][] newtypes =
-          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
-              {StorageType.ARCHIVE, StorageType.ARCHIVE},
-              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+          new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+              {StorageType.DISK, StorageType.ARCHIVE},
+              {StorageType.DISK, StorageType.ARCHIVE}};
       startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
           storagesPerDatanode, capacity, hdfsCluster);
 
@@ -314,7 +319,7 @@ public class TestStoragePolicySatisfier {
       writeContent(subFile2);
 
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(subDir), "ONE_SSD");
+      dfs.setStoragePolicy(new Path(subDir), ONE_SSD);
 
       StorageType[][] newtypes =
           new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
@@ -418,7 +423,7 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), "COLD");
+      dfs.setStoragePolicy(new Path(file), COLD);
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -463,7 +468,7 @@ public class TestStoragePolicySatisfier {
     try {
       createCluster();
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), "COLD");
+      dfs.setStoragePolicy(new Path(file), COLD);
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -533,7 +538,7 @@ public class TestStoragePolicySatisfier {
     final String file1 = createFileAndSimulateFavoredNodes(2);
 
     // Change policy to COLD
-    dfs.setStoragePolicy(new Path(file1), "COLD");
+    dfs.setStoragePolicy(new Path(file1), COLD);
     FSNamesystem namesystem = hdfsCluster.getNamesystem();
     INode inode = namesystem.getFSDirectory().getINode(file1);
 
@@ -594,7 +599,7 @@ public class TestStoragePolicySatisfier {
       writeContent(file, (short) 5);
 
       // Change policy to COLD
-      dfs.setStoragePolicy(new Path(file), "COLD");
+      dfs.setStoragePolicy(new Path(file), COLD);
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -633,7 +638,7 @@ public class TestStoragePolicySatisfier {
       writeContent(file);
 
       // Change policy to ONE_SSD
-      dfs.setStoragePolicy(new Path(file), "ONE_SSD");
+      dfs.setStoragePolicy(new Path(file), ONE_SSD);
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -688,6 +693,77 @@ public class TestStoragePolicySatisfier {
     }
   }
 
+  /**
+   * Tests that movements should not be assigned when there is no space in
+   * target DN.
+   */
+  @Test(timeout = 300000)
+  public void testChooseInSameDatanodeWithONESSDShouldNotChooseIfNoSpace()
+      throws Exception {
+    StorageType[][] diskTypes =
+        new StorageType[][]{{StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.SSD},
+            {StorageType.DISK, StorageType.DISK}};
+    config.setLong("dfs.block.size", 2 * DEFAULT_BLOCK_SIZE);
+    long dnCapacity = 1024 * DEFAULT_BLOCK_SIZE + (2 * DEFAULT_BLOCK_SIZE - 1);
+    try {
+      hdfsCluster = startCluster(config, diskTypes, numOfDatanodes,
+          storagesPerDatanode, dnCapacity);
+      dfs = hdfsCluster.getFileSystem();
+      writeContent(file);
+
+      // Change policy to ONE_SSD
+      dfs.setStoragePolicy(new Path(file), ONE_SSD);
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+      Path filePath = new Path("/testChooseInSameDatanode");
+      final FSDataOutputStream out =
+          dfs.create(filePath, false, 100, (short) 1, 2 * DEFAULT_BLOCK_SIZE);
+      try {
+        dfs.setStoragePolicy(filePath, ONE_SSD);
+        // Try to fill up SSD part by writing content
+        long remaining = dfs.getStatus().getRemaining() / (3 * 2);
+        for (int i = 0; i < remaining; i++) {
+          out.write(i);
+        }
+      } finally {
+        out.close();
+      }
+      hdfsCluster.triggerHeartbeats();
+      ArrayList<DataNode> dataNodes = hdfsCluster.getDataNodes();
+      // Temporarily disable heart beats, so that we can assert whether any
+      // items schedules for DNs even though DN's does not have space to write.
+      // Disabling heart beats can keep scheduled items on DatanodeDescriptor
+      // itself.
+      for (DataNode dataNode : dataNodes) {
+        DataNodeTestUtils.setHeartbeatsDisabledForTests(dataNode, true);
+      }
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+
+      // Wait for items to be processed
+      waitForAttemptedItems(1, 30000);
+
+      // Make sure no items assigned for movements
+      Set<DatanodeDescriptor> dns = hdfsCluster.getNamesystem()
+          .getBlockManager().getDatanodeManager().getDatanodes();
+      for (DatanodeDescriptor dd : dns) {
+        assertNull(dd.getBlocksToMoveStorages());
+      }
+
+      // Enable heart beats now
+      for (DataNode dataNode : dataNodes) {
+        DataNodeTestUtils.setHeartbeatsDisabledForTests(dataNode, false);
+      }
+      hdfsCluster.triggerHeartbeats();
+
+      DFSTestUtil.waitExpectedStorageType(file, StorageType.DISK, 3, 30000,
+          dfs);
+      DFSTestUtil.waitExpectedStorageType(file, StorageType.SSD, 0, 30000, dfs);
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
       throws IOException {
     ArrayList<DataNode> dns = hdfsCluster.getDataNodes();
@@ -769,8 +845,8 @@ public class TestStoragePolicySatisfier {
     // write to DISK
     final FSDataOutputStream out = dfs.create(new Path(fileName),
         replicatonFactor);
-    for (int i = 0; i < 1000; i++) {
-      out.writeChars("t");
+    for (int i = 0; i < 1024; i++) {
+      out.write(i);
     }
     out.close();
   }


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


[09/50] [abbrv] hadoop git commit: HDFS-11068: [SPS]: Provide unique trackID to track the block movement sends to coordinator. Contributed by Rakesh R

Posted by ra...@apache.org.
HDFS-11068: [SPS]: Provide unique trackID to track the block movement sends to coordinator. 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/bfb1a50c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bfb1a50c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bfb1a50c

Branch: refs/heads/HDFS-10285
Commit: bfb1a50c9d1e448e75f5be0ef243159875680bc6
Parents: 9ddfbe1
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Fri Nov 11 01:17:50 2016 -0800
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:20:22 2017 +0530

----------------------------------------------------------------------
 .../blockmanagement/DatanodeDescriptor.java     | 31 +++++--
 .../server/blockmanagement/DatanodeManager.java | 16 ++--
 .../BlockStorageMovementInfosBatch.java         | 61 +++++++++++++
 .../server/namenode/StoragePolicySatisfier.java |  5 +-
 .../protocol/BlocksStorageMovementResult.java   |  6 ++
 .../namenode/TestStoragePolicySatisfier.java    | 95 +++++++++++++++-----
 6 files changed, 174 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfb1a50c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 1db002b..9d3b544 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -41,9 +41,9 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.namenode.BlockStorageMovementInfosBatch;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
@@ -207,8 +207,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final LightWeightHashSet<Block> invalidateBlocks =
       new LightWeightHashSet<>();
 
-  /** A queue of blocks for moving its storage placements by this datanode. */
-  private final Queue<List<BlockMovingInfo>> storageMovementBlocks =
+  /**
+   * A queue of blocks corresponding to trackID for moving its storage
+   * placements by this datanode.
+   */
+  private final Queue<BlockStorageMovementInfosBatch> storageMovementBlocks =
       new LinkedList<>();
 
   /* Variables for maintaining number of blocks scheduled to be written to
@@ -1025,18 +1028,30 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Add the block infos which needs to move its storage locations.
    *
+   * @param trackID
+   *          - unique identifier which will be used for tracking the given set
+   *          of blocks movement completion.
    * @param storageMismatchedBlocks
    *          - storage mismatched block infos
    */
-  public void addBlocksToMoveStorage(
+  public void addBlocksToMoveStorage(long trackID,
       List<BlockMovingInfo> storageMismatchedBlocks) {
-    storageMovementBlocks.offer(storageMismatchedBlocks);
+    synchronized (storageMovementBlocks) {
+      storageMovementBlocks.offer(
+          new BlockStorageMovementInfosBatch(trackID, storageMismatchedBlocks));
+    }
   }
 
   /**
-   * @return block infos which needs to move its storage locations.
+   * @return block infos which needs to move its storage locations. This returns
+   *         list of blocks under one trackId.
    */
-  public List<BlockMovingInfo> getBlocksToMoveStorages() {
-    return storageMovementBlocks.poll();
+  public BlockStorageMovementInfosBatch getBlocksToMoveStorages() {
+    synchronized (storageMovementBlocks) {
+      // TODO: Presently returning the list of blocks under one trackId.
+      // Need to limit the list of items into small batches with in trackId
+      // itself if blocks are many(For example: a file contains many blocks).
+      return storageMovementBlocks.poll();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfb1a50c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 6a5cfbe..048b68f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.common.Util;
+import org.apache.hadoop.hdfs.server.namenode.BlockStorageMovementInfosBatch;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
@@ -47,7 +48,6 @@ import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
@@ -1738,16 +1738,14 @@ public class DatanodeManager {
     }
 
     // check pending block storage movement tasks
-    List<BlockMovingInfo> pendingBlockMovementList = nodeinfo
+    BlockStorageMovementInfosBatch blkStorageMovementInfosBatch = nodeinfo
         .getBlocksToMoveStorages();
-    if (pendingBlockMovementList != null) {
-      // TODO: trackID is used to track the block movement sends to coordinator
-      // datanode. Need to implement tracking logic. Temporarily, using a
-      // constant value -1.
-      long trackID = -1;
+
+    if (blkStorageMovementInfosBatch != null) {
       cmds.add(new BlockStorageMovementCommand(
-          DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT, trackID, blockPoolId,
-          pendingBlockMovementList));
+          DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT,
+          blkStorageMovementInfosBatch.getTrackID(), blockPoolId,
+          blkStorageMovementInfosBatch.getBlockMovingInfo()));
     }
 
     if (!cmds.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfb1a50c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java
new file mode 100644
index 0000000..a790c13
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementInfosBatch.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.List;
+
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+
+/**
+ * This class represents a batch of blocks under one trackId which needs to move
+ * its storage locations to satisfy the storage policy.
+ */
+public class BlockStorageMovementInfosBatch {
+  private long trackID;
+  private List<BlockMovingInfo> blockMovingInfos;
+
+  /**
+   * Constructor to create the block storage movement infos batch.
+   *
+   * @param trackID
+   *          - unique identifier which will be used for tracking the given set
+   *          of blocks movement.
+   * @param blockMovingInfos
+   *          - list of block to storage infos.
+   */
+  public BlockStorageMovementInfosBatch(long trackID,
+      List<BlockMovingInfo> blockMovingInfos) {
+    this.trackID = trackID;
+    this.blockMovingInfos = blockMovingInfos;
+  }
+
+  public long getTrackID() {
+    return trackID;
+  }
+
+  public List<BlockMovingInfo> getBlockMovingInfo() {
+    return blockMovingInfos;
+  }
+
+  @Override
+  public String toString() {
+    return new StringBuilder().append("BlockStorageMovementInfosBatch(\n  ")
+        .append("TrackID: ").append(trackID).append("  BlockMovingInfos: ")
+        .append(blockMovingInfos).append(")").toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfb1a50c/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 6fa9302..4967a89 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
@@ -217,7 +217,10 @@ public class StoragePolicySatisfier implements Runnable {
       // chances, then we can just retry limited number of times and exit.
       return;
     }
-    coordinatorNode.addBlocksToMoveStorage(blockMovingInfos);
+
+    // 'BlockCollectionId' is used as the tracking ID. All the blocks under this
+    // blockCollectionID will be added to this datanode.
+    coordinatorNode.addBlocksToMoveStorage(blockCollectionID, blockMovingInfos);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfb1a50c/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 1afba34..713b83b 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
@@ -61,4 +61,10 @@ public class BlocksStorageMovementResult {
     return status;
   }
 
+  @Override
+  public String toString() {
+    return new StringBuilder().append("BlocksStorageMovementResult(\n  ")
+        .append("track id: ").append(trackId).append("  status: ")
+        .append(status).append(")").toString();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfb1a50c/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 6f5c717..e84052f 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -54,15 +56,15 @@ public class TestStoragePolicySatisfier {
   final private int storagesPerDatanode = 2;
   final private long capacity = 2 * 256 * 1024 * 1024;
   final private String file = "/testMoveWhenStoragePolicyNotSatisfying";
-  private DistributedFileSystem distributedFS = null;
+  private DistributedFileSystem dfs = null;
 
   @Before
   public void setUp() throws IOException {
     config.setLong("dfs.block.size", 1024);
     hdfsCluster = startCluster(config, allDiskTypes, numOfDatanodes,
         storagesPerDatanode, capacity);
-    distributedFS = hdfsCluster.getFileSystem();
-    writeContent(distributedFS, file);
+    dfs = hdfsCluster.getFileSystem();
+    writeContent(file);
   }
 
   @Test(timeout = 300000)
@@ -71,7 +73,7 @@ public class TestStoragePolicySatisfier {
 
     try {
       // Change policy to ALL_SSD
-      distributedFS.setStoragePolicy(new Path(file), "COLD");
+      dfs.setStoragePolicy(new Path(file), "COLD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -86,8 +88,7 @@ public class TestStoragePolicySatisfier {
 
       hdfsCluster.triggerHeartbeats();
       // Wait till namenode notified about the block location details
-      waitExpectedStorageType(file, StorageType.ARCHIVE, distributedFS, 3,
-          30000);
+      waitExpectedStorageType(file, StorageType.ARCHIVE, 3, 30000);
     } finally {
       hdfsCluster.shutdown();
     }
@@ -98,7 +99,7 @@ public class TestStoragePolicySatisfier {
       throws Exception {
     try {
       // Change policy to ALL_SSD
-      distributedFS.setStoragePolicy(new Path(file), "ALL_SSD");
+      dfs.setStoragePolicy(new Path(file), "ALL_SSD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -115,7 +116,7 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 3, 30000);
+      waitExpectedStorageType(file, StorageType.SSD, 3, 30000);
     } finally {
       hdfsCluster.shutdown();
     }
@@ -126,7 +127,7 @@ public class TestStoragePolicySatisfier {
       throws Exception {
     try {
       // Change policy to ONE_SSD
-      distributedFS.setStoragePolicy(new Path(file), "ONE_SSD");
+      dfs.setStoragePolicy(new Path(file), "ONE_SSD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -141,8 +142,8 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 1, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, distributedFS, 2, 30000);
+      waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
     } finally {
       hdfsCluster.shutdown();
     }
@@ -156,7 +157,7 @@ public class TestStoragePolicySatisfier {
   public void testPerTrackIdBlocksStorageMovementResults() throws Exception {
     try {
       // Change policy to ONE_SSD
-      distributedFS.setStoragePolicy(new Path(file), "ONE_SSD");
+      dfs.setStoragePolicy(new Path(file), "ONE_SSD");
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -171,8 +172,8 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
 
       // Wait till the block is moved to SSD areas
-      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 1, 30000);
-      waitExpectedStorageType(file, StorageType.DISK, distributedFS, 2, 30000);
+      waitExpectedStorageType(file, StorageType.SSD, 1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, 2, 30000);
 
       waitForBlocksMovementResult(1, 30000);
     } finally {
@@ -180,7 +181,58 @@ public class TestStoragePolicySatisfier {
     }
   }
 
-  private void waitForBlocksMovementResult(int expectedResultsCount,
+  /**
+   * Tests to verify that multiple files are giving to satisfy storage policy
+   * and should work well altogether.
+   */
+  @Test(timeout = 300000)
+  public void testMultipleFilesForSatisfyStoragePolicy() throws Exception {
+    List<String> files = new ArrayList<>();
+    files.add(file);
+
+    // Creates 4 more files. Send all of them for satisfying the storage policy
+    // together.
+    for (int i = 0; i < 4; i++) {
+      String file1 = "/testMoveWhenStoragePolicyNotSatisfying_" + i;
+      files.add(file1);
+      writeContent(file1);
+    }
+
+    try {
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      List<Long> blockCollectionIds = new ArrayList<>();
+      // Change policy to ONE_SSD
+      for (String fileName : files) {
+        dfs.setStoragePolicy(new Path(fileName), "ONE_SSD");
+        INode inode = namesystem.getFSDirectory().getINode(fileName);
+        blockCollectionIds.add(inode.getId());
+      }
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+      for (long inodeId : blockCollectionIds) {
+        namesystem.getBlockManager().satisfyStoragePolicy(inodeId);
+      }
+      hdfsCluster.triggerHeartbeats();
+
+      for (String fileName : files) {
+        // Wait till the block is moved to SSD areas
+        waitExpectedStorageType(fileName, StorageType.SSD, 1, 30000);
+        waitExpectedStorageType(fileName, StorageType.DISK, 2, 30000);
+      }
+
+      waitForBlocksMovementResult(blockCollectionIds.size(), 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  private void waitForBlocksMovementResult(long expectedBlkMovResultsCount,
       int timeout) throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
     final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
@@ -188,16 +240,15 @@ public class TestStoragePolicySatisfier {
       @Override
       public Boolean get() {
         LOG.info("expectedResultsCount={} actualResultsCount={}",
-            expectedResultsCount,
+            expectedBlkMovResultsCount,
             sps.getAttemptedItemsMonitor().resultsCount());
-        return expectedResultsCount == sps.getAttemptedItemsMonitor()
-            .resultsCount();
+        return sps.getAttemptedItemsMonitor()
+            .resultsCount() == expectedBlkMovResultsCount;
       }
     }, 100, timeout);
   }
 
-  private void writeContent(final DistributedFileSystem dfs,
-      final String fileName) throws IOException {
+  private void writeContent(final String fileName) throws IOException {
     // write to DISK
     final FSDataOutputStream out = dfs.create(new Path(fileName));
     for (int i = 0; i < 1000; i++) {
@@ -243,8 +294,8 @@ public class TestStoragePolicySatisfier {
   // Check whether the Block movement has been successfully completed to satisfy
   // the storage policy for the given file.
   private void waitExpectedStorageType(final String fileName,
-      final StorageType expectedStorageType, final DistributedFileSystem dfs,
-      int expectedStorageCount, int timeout) throws Exception {
+      final StorageType expectedStorageType, int expectedStorageCount,
+      int timeout) throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {


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


[38/50] [abbrv] hadoop git commit: HDFS-12141: [SPS]: Fix checkstyle warnings. Contributed by Rakesh R.

Posted by ra...@apache.org.
HDFS-12141: [SPS]: Fix checkstyle warnings. 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/c2c9296e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c2c9296e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c2c9296e

Branch: refs/heads/HDFS-10285
Commit: c2c9296e783add4d553830fbebf67ba3d19d1bf8
Parents: 5eed1ce
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Mon Jul 17 10:24:06 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:40 2017 +0530

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/BlockManager.java       |  2 +-
 .../server/datanode/StoragePolicySatisfyWorker.java     |  6 +++---
 .../hdfs/server/namenode/StoragePolicySatisfier.java    |  6 +++---
 .../hadoop/hdfs/server/protocol/DatanodeProtocol.java   |  5 ++---
 .../org/apache/hadoop/hdfs/server/mover/TestMover.java  |  7 ++++---
 .../server/namenode/TestStoragePolicySatisfier.java     | 12 ++++++------
 6 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2c9296e/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 37d6c1b..8349465 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
@@ -432,7 +432,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   private final BlockIdManager blockIdManager;
 
-  /** For satisfying block storage policies */
+  /** For satisfying block storage policies. */
   private final StoragePolicySatisfier sps;
   private final BlockStorageMovementNeeded storageMovementNeeded =
       new BlockStorageMovementNeeded();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2c9296e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index f4f97dd..196cd58 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -329,7 +329,7 @@ public class StoragePolicySatisfyWorker {
   /**
    * Block movement status code.
    */
-  public static enum BlockMovementStatus {
+  public enum BlockMovementStatus {
     /** Success. */
     DN_BLK_STORAGE_MOVEMENT_SUCCESS(0),
     /**
@@ -343,7 +343,7 @@ public class StoragePolicySatisfyWorker {
 
     private final int code;
 
-    private BlockMovementStatus(int code) {
+    BlockMovementStatus(int code) {
       this.code = code;
     }
 
@@ -365,7 +365,7 @@ public class StoragePolicySatisfyWorker {
     private final DatanodeInfo target;
     private final BlockMovementStatus status;
 
-    public BlockMovementResult(long trackId, long blockId,
+    BlockMovementResult(long trackId, long blockId,
         DatanodeInfo target, BlockMovementStatus status) {
       this.trackId = trackId;
       this.blockId = blockId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2c9296e/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 00b4cd0..af3b7f2 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
@@ -714,10 +714,10 @@ public class StoragePolicySatisfier implements Runnable {
   }
 
   private static class StorageTypeNodePair {
-    public StorageType storageType = null;
-    public DatanodeDescriptor dn = null;
+    private StorageType storageType = null;
+    private DatanodeDescriptor dn = null;
 
-    public StorageTypeNodePair(StorageType storageType, DatanodeDescriptor dn) {
+    StorageTypeNodePair(StorageType storageType, DatanodeDescriptor dn) {
       this.storageType = storageType;
       this.dn = dn;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2c9296e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 892efb3..5e1f148 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -79,9 +79,8 @@ public interface DatanodeProtocol {
   final static int DNA_CACHE = 9;      // cache blocks
   final static int DNA_UNCACHE = 10;   // uncache blocks
   final static int DNA_ERASURE_CODING_RECONSTRUCTION = 11; // erasure coding reconstruction command
-  final static int DNA_BLOCK_STORAGE_MOVEMENT = 12; // block storage movement command
-  final static int DNA_DROP_SPS_WORK_COMMAND = 13; // block storage movement
-                                                   // command
+  int DNA_BLOCK_STORAGE_MOVEMENT = 12; // block storage movement command
+  int DNA_DROP_SPS_WORK_COMMAND = 13; // drop sps work command
 
   /** 
    * Register Datanode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2c9296e/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 de7dd29..17fd37c 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
@@ -125,9 +125,10 @@ public class TestMover {
       nnMap.put(nn, null);
     }
 
-    final List<NameNodeConnector> nncs = NameNodeConnector.newNameNodeConnectors(
-        nnMap, Mover.class.getSimpleName(), HdfsServerConstants.MOVER_ID_PATH, conf,
-        NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
+    final List<NameNodeConnector> nncs = NameNodeConnector.
+        newNameNodeConnectors(nnMap, Mover.class.getSimpleName(),
+            HdfsServerConstants.MOVER_ID_PATH, conf,
+            NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
     return new Mover(nncs.get(0), conf, new AtomicInteger(0), new HashMap<>());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2c9296e/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 10ceae7..7f96003 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
@@ -496,8 +496,8 @@ public class TestStoragePolicySatisfier {
       namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
       hdfsCluster.triggerHeartbeats();
 
-      // No block movement will be scheduled as there is no target node available
-      // with the required storage type.
+      // No block movement will be scheduled as there is no target node
+      // available with the required storage type.
       waitForAttemptedItems(1, 30000);
       DFSTestUtil.waitExpectedStorageType(
           file, StorageType.DISK, 3, 30000, dfs);
@@ -1174,14 +1174,14 @@ public class TestStoragePolicySatisfier {
 
   private void startAdditionalDNs(final Configuration conf,
       int newNodesRequired, int existingNodesNum, StorageType[][] newTypes,
-      int storagesPerDatanode, long capacity, final MiniDFSCluster cluster)
+      int storagesPerDn, long nodeCapacity, final MiniDFSCluster cluster)
           throws IOException {
     long[][] capacities;
     existingNodesNum += newNodesRequired;
-    capacities = new long[newNodesRequired][storagesPerDatanode];
+    capacities = new long[newNodesRequired][storagesPerDn];
     for (int i = 0; i < newNodesRequired; i++) {
-      for (int j = 0; j < storagesPerDatanode; j++) {
-        capacities[i][j] = capacity;
+      for (int j = 0; j < storagesPerDn; j++) {
+        capacities[i][j] = nodeCapacity;
       }
     }
 


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


[44/50] [abbrv] hadoop git commit: HDFS-12570: [SPS]: Refactor Co-ordinator datanode logic to track the block storage movements. Contributed by Rakesh R.

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 57e9f94..70219f6 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
@@ -203,11 +203,11 @@ public class TestStoragePolicySatisfier {
   }
 
   /**
-   * Tests to verify that the block storage movement results will be propagated
+   * Tests to verify that the block storage movement report will be propagated
    * to Namenode via datanode heartbeat.
    */
   @Test(timeout = 300000)
-  public void testPerTrackIdBlocksStorageMovementResults() throws Exception {
+  public void testBlksStorageMovementAttemptFinishedReport() throws Exception {
     try {
       createCluster();
       // Change policy to ONE_SSD
@@ -229,7 +229,7 @@ public class TestStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           file, StorageType.DISK, 2, 30000, dfs);
 
-      waitForBlocksMovementResult(1, 30000);
+      waitForBlocksMovementAttemptReport(1, 30000);
     } finally {
       shutdownCluster();
     }
@@ -276,7 +276,7 @@ public class TestStoragePolicySatisfier {
             fileName, StorageType.DISK, 2, 30000, dfs);
       }
 
-      waitForBlocksMovementResult(files.size(), 30000);
+      waitForBlocksMovementAttemptReport(files.size(), 30000);
     } finally {
       shutdownCluster();
     }
@@ -457,7 +457,7 @@ public class TestStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           file, StorageType.DISK, 2, 30000, dfs);
 
-      waitForBlocksMovementResult(1, 30000);
+      waitForBlocksMovementAttemptReport(1, 30000);
     } finally {
       shutdownCluster();
     }
@@ -630,7 +630,7 @@ public class TestStoragePolicySatisfier {
       // No block movement will be scheduled as there is no target node
       // available with the required storage type.
       waitForAttemptedItems(1, 30000);
-      waitForBlocksMovementResult(1, 30000);
+      waitForBlocksMovementAttemptReport(1, 30000);
       DFSTestUtil.waitExpectedStorageType(
           file1, StorageType.ARCHIVE, 1, 30000, dfs);
       DFSTestUtil.waitExpectedStorageType(
@@ -691,7 +691,7 @@ public class TestStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           file, StorageType.DISK, 3, 30000, dfs);
 
-      waitForBlocksMovementResult(1, 30000);
+      waitForBlocksMovementAttemptReport(1, 30000);
     } finally {
       shutdownCluster();
     }
@@ -871,7 +871,7 @@ public class TestStoragePolicySatisfier {
       Set<DatanodeDescriptor> dns = hdfsCluster.getNamesystem()
           .getBlockManager().getDatanodeManager().getDatanodes();
       for (DatanodeDescriptor dd : dns) {
-        assertNull(dd.getBlocksToMoveStorages());
+        assertNull(dd.getBlocksToMoveStorages(1));
       }
 
       // Enable heart beats now
@@ -1224,7 +1224,7 @@ public class TestStoragePolicySatisfier {
   /**
    * Test SPS for batch processing.
    */
-  @Test(timeout = 300000)
+  @Test(timeout = 3000000)
   public void testBatchProcessingForSPSDirectory() throws Exception {
     try {
       StorageType[][] diskTypes = new StorageType[][] {
@@ -1252,7 +1252,7 @@ public class TestStoragePolicySatisfier {
         DFSTestUtil.waitExpectedStorageType(fileName, StorageType.ARCHIVE, 2,
             30000, dfs);
       }
-      waitForBlocksMovementResult(files.size(), 30000);
+      waitForBlocksMovementAttemptReport(files.size(), 30000);
       String expectedLogMessage = "StorageMovementNeeded queue remaining"
           + " capacity is zero";
       assertTrue("Log output does not contain expected log message: "
@@ -1268,7 +1268,7 @@ public class TestStoragePolicySatisfier {
    *  1. Delete /root when traversing Q
    *  2. U, R, S should not be in queued.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testTraverseWhenParentDeleted() throws Exception {
     StorageType[][] diskTypes = new StorageType[][] {
         {StorageType.DISK, StorageType.ARCHIVE},
@@ -1330,7 +1330,7 @@ public class TestStoragePolicySatisfier {
    *  1. Delete L when traversing Q
    *  2. E, M, U, R, S should not be in queued.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testTraverseWhenRootParentDeleted() throws Exception {
     StorageType[][] diskTypes = new StorageType[][] {
         {StorageType.DISK, StorageType.ARCHIVE},
@@ -1387,6 +1387,82 @@ public class TestStoragePolicySatisfier {
     dfs.delete(new Path("/root"), true);
   }
 
+  /**
+   * Test storage move blocks while under replication block tasks exists in the
+   * system. So, both will share the max transfer streams.
+   *
+   * 1. Create cluster with 3 datanode.
+   * 2. Create 20 files with 2 replica.
+   * 3. Start 2 more DNs with DISK & SSD types
+   * 4. SetReplication factor for the 1st 10 files to 4 to trigger replica task
+   * 5. Set policy to SSD to the 2nd set of files from 11-20
+   * 6. Call SPS for 11-20 files to trigger move block tasks to new DNs
+   * 7. Wait for the under replica and SPS tasks completion
+   */
+  @Test(timeout = 300000)
+  public void testMoveBlocksWithUnderReplicatedBlocks() throws Exception {
+    try {
+      config.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 3);
+      config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+      config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
+          true);
+      config.set(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
+          "3000");
+      config.setBoolean(DFSConfigKeys
+          .DFS_STORAGE_POLICY_SATISFIER_SHARE_EQUAL_REPLICA_MAX_STREAMS_KEY,
+          true);
+
+      StorageType[][] storagetypes = new StorageType[][] {
+          {StorageType.ARCHIVE, StorageType.DISK},
+          {StorageType.ARCHIVE, StorageType.DISK}};
+      hdfsCluster = new MiniDFSCluster.Builder(config).numDataNodes(2)
+          .storageTypes(storagetypes).build();
+      hdfsCluster.waitActive();
+      dfs = hdfsCluster.getFileSystem();
+
+      // Below files will be used for pending replication block tasks.
+      for (int i=1; i<=20; i++){
+        Path filePath = new Path("/file" + i);
+        DFSTestUtil.createFile(dfs, filePath, DEFAULT_BLOCK_SIZE * 5, (short) 2,
+            0);
+      }
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.DISK, StorageType.SSD},
+              {StorageType.DISK, StorageType.SSD}};
+      startAdditionalDNs(config, 2, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      // increase replication factor to 4 for the first 10 files and thus
+      // initiate replica tasks
+      for (int i=1; i<=10; i++){
+        Path filePath = new Path("/file" + i);
+        dfs.setReplication(filePath, (short) 4);
+      }
+
+      // invoke SPS for 11-20 files
+      for (int i = 11; i <= 20; i++) {
+        Path filePath = new Path("/file" + i);
+        dfs.setStoragePolicy(filePath, "ALL_SSD");
+        dfs.satisfyStoragePolicy(filePath);
+      }
+
+      for (int i = 1; i <= 10; i++) {
+        Path filePath = new Path("/file" + i);
+        DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+            StorageType.DISK, 4, 30000, hdfsCluster.getFileSystem());
+      }
+      for (int i = 11; i <= 20; i++) {
+        Path filePath = new Path("/file" + i);
+        DFSTestUtil.waitExpectedStorageType(filePath.toString(),
+            StorageType.SSD, 2, 30000, hdfsCluster.getFileSystem());
+      }
+    } finally {
+      shutdownCluster();
+    }
+  }
+
   private static void createDirectoryTree(DistributedFileSystem dfs)
       throws Exception {
     // tree structure
@@ -1514,18 +1590,19 @@ public class TestStoragePolicySatisfier {
     }, 100, timeout);
   }
 
-  private void waitForBlocksMovementResult(long expectedBlkMovResultsCount,
-      int timeout) throws TimeoutException, InterruptedException {
+  private void waitForBlocksMovementAttemptReport(
+      long expectedMovementFinishedBlocksCount, int timeout)
+          throws TimeoutException, InterruptedException {
     BlockManager blockManager = hdfsCluster.getNamesystem().getBlockManager();
     final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        LOG.info("expectedResultsCount={} actualResultsCount={}",
-            expectedBlkMovResultsCount,
-            sps.getAttemptedItemsMonitor().resultsCount());
-        return sps.getAttemptedItemsMonitor()
-            .resultsCount() == expectedBlkMovResultsCount;
+        LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
+            expectedMovementFinishedBlocksCount,
+            sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount());
+        return sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount()
+            >= expectedMovementFinishedBlocksCount;
       }
     }, 100, timeout);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42b22d97/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 fc5d0a5..154ddae 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
@@ -180,7 +180,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
       LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
       cluster.triggerHeartbeats();
 
-      waitForBlocksMovementResult(cluster, 1, 60000);
+      waitForBlocksMovementAttemptReport(cluster, 9, 60000);
       // verify storage types and locations
       waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 9,
           9, 60000);
@@ -290,7 +290,7 @@ public class TestStoragePolicySatisfierWithStripedFile {
       LOG.info("Sets storage policy to COLD and invoked satisfyStoragePolicy");
       cluster.triggerHeartbeats();
 
-      waitForBlocksMovementResult(cluster, 1, 60000);
+      waitForBlocksMovementAttemptReport(cluster, 5, 60000);
       waitForAttemptedItems(cluster, 1, 30000);
       // verify storage types and locations.
       waitExpectedStorageType(cluster, fooFile, fileLen, StorageType.ARCHIVE, 5,
@@ -556,10 +556,10 @@ public class TestStoragePolicySatisfierWithStripedFile {
     }, 100, timeout);
   }
 
-  // Check whether the block movement result has been arrived at the
+  // Check whether the block movement attempt report has been arrived at the
   // Namenode(SPS).
-  private void waitForBlocksMovementResult(MiniDFSCluster cluster,
-      long expectedBlkMovResultsCount, int timeout)
+  private void waitForBlocksMovementAttemptReport(MiniDFSCluster cluster,
+      long expectedMovementFinishedBlocksCount, int timeout)
           throws TimeoutException, InterruptedException {
     BlockManager blockManager = cluster.getNamesystem().getBlockManager();
     final StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
@@ -568,11 +568,11 @@ public class TestStoragePolicySatisfierWithStripedFile {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        LOG.info("expectedResultsCount={} actualResultsCount={}",
-            expectedBlkMovResultsCount,
-            sps.getAttemptedItemsMonitor().resultsCount());
-        return sps.getAttemptedItemsMonitor()
-            .resultsCount() == expectedBlkMovResultsCount;
+        LOG.info("MovementFinishedBlocks: expectedCount={} actualCount={}",
+            expectedMovementFinishedBlocksCount,
+            sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount());
+        return sps.getAttemptedItemsMonitor().getMovementFinishedBlocksCount()
+            >= expectedMovementFinishedBlocksCount;
       }
     }, 100, timeout);
   }


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


[43/50] [abbrv] hadoop git commit: HDFS-12291: [SPS]: Provide a mechanism to recursively iterate and satisfy storage policy of all the files under the given dir. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-12291: [SPS]: Provide a mechanism to recursively iterate and satisfy storage policy of all the files under the given dir. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: e64a4280902bea53ec0642f8f8f59ff292a8ca49
Parents: b58239d
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Sat Sep 30 06:31:52 2017 -0700
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:31:08 2017 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  22 +-
 .../BlockStorageMovementAttemptedItems.java     |   8 +-
 .../namenode/BlockStorageMovementNeeded.java    | 277 +++++++--
 .../hdfs/server/namenode/FSTreeTraverser.java   | 313 ++++++++++
 .../server/namenode/ReencryptionHandler.java    | 618 ++++++++-----------
 .../server/namenode/ReencryptionUpdater.java    |   2 +-
 .../server/namenode/StoragePolicySatisfier.java |  43 +-
 .../src/main/resources/hdfs-default.xml         |  23 +
 .../src/site/markdown/ArchivalStorage.md        |   3 +-
 .../TestBlockStorageMovementAttemptedItems.java |   2 +-
 .../TestPersistentStoragePolicySatisfier.java   |   8 +-
 .../hdfs/server/namenode/TestReencryption.java  |   3 -
 .../namenode/TestReencryptionHandler.java       |  10 +-
 .../namenode/TestStoragePolicySatisfier.java    | 377 ++++++++++-
 15 files changed, 1260 insertions(+), 457 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 5c5a3d9..a73d399 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
@@ -574,6 +574,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.storage.policy.satisfier.enabled";
   public static final boolean DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT =
       false;
+  public static final String  DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY =
+      "dfs.storage.policy.satisfier.queue.limit";
+  public static final int  DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT =
+      1000;
+  public static final String DFS_SPS_WORK_MULTIPLIER_PER_ITERATION =
+      "dfs.storage.policy.satisfier.work.multiplier.per.iteration";
+  public static final int DFS_SPS_WORK_MULTIPLIER_PER_ITERATION_DEFAULT =
+      1;
   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/e64a4280/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 7465853..570b85d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -1422,7 +1422,27 @@ public class DFSUtil {
         "It should be a positive, non-zero integer value.");
     return blocksReplWorkMultiplier;
   }
-  
+
+  /**
+   * Get DFS_SPS_WORK_MULTIPLIER_PER_ITERATION from
+   * configuration.
+   *
+   * @param conf Configuration
+   * @return Value of DFS_SPS_WORK_MULTIPLIER_PER_ITERATION
+   */
+  public static int getSPSWorkMultiplier(Configuration conf) {
+    int spsWorkMultiplier = conf
+        .getInt(
+            DFSConfigKeys.DFS_SPS_WORK_MULTIPLIER_PER_ITERATION,
+            DFSConfigKeys.DFS_SPS_WORK_MULTIPLIER_PER_ITERATION_DEFAULT);
+    Preconditions.checkArgument(
+        (spsWorkMultiplier > 0),
+        DFSConfigKeys.DFS_SPS_WORK_MULTIPLIER_PER_ITERATION +
+        " = '" + spsWorkMultiplier + "' is invalid. " +
+        "It should be a positive, non-zero integer value.");
+    return spsWorkMultiplier;
+  }
+
   /**
    * Get SPNEGO keytab Key from configuration
    * 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 278b62b..549819f 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
@@ -101,7 +101,7 @@ public class BlockStorageMovementAttemptedItems {
   public void add(ItemInfo itemInfo, boolean allBlockLocsAttemptedToSatisfy) {
     synchronized (storageMovementAttemptedItems) {
       AttemptedItemInfo attemptedItemInfo = new AttemptedItemInfo(
-          itemInfo.getRootId(), itemInfo.getTrackId(), monotonicNow(),
+          itemInfo.getStartId(), itemInfo.getTrackId(), monotonicNow(),
           allBlockLocsAttemptedToSatisfy);
       storageMovementAttemptedItems.put(itemInfo.getTrackId(),
           attemptedItemInfo);
@@ -260,7 +260,7 @@ public class BlockStorageMovementAttemptedItems {
           synchronized (storageMovementAttemptedResults) {
             if (!isExistInResult(blockCollectionID)) {
               ItemInfo candidate = new ItemInfo(
-                  itemInfo.getRootId(), blockCollectionID);
+                  itemInfo.getStartId(), blockCollectionID);
               blockStorageMovementNeeded.add(candidate);
               iter.remove();
               LOG.info("TrackID: {} becomes timed out and moved to needed "
@@ -315,7 +315,7 @@ public class BlockStorageMovementAttemptedItems {
           // blockStorageMovementNeeded#removeIteamTrackInfo() for cleaning
           // the xAttr
           ItemInfo itemInfo = new ItemInfo((attemptedItemInfo != null)
-              ? attemptedItemInfo.getRootId() : trackId, trackId);
+              ? attemptedItemInfo.getStartId() : trackId, trackId);
           switch (status) {
           case FAILURE:
             if (attemptedItemInfo != null) {
@@ -345,7 +345,7 @@ public class BlockStorageMovementAttemptedItems {
             if (attemptedItemInfo != null) {
               if (!attemptedItemInfo.isAllBlockLocsAttemptedToSatisfy()) {
                 blockStorageMovementNeeded
-                    .add(new ItemInfo(attemptedItemInfo.getRootId(), trackId));
+                    .add(new ItemInfo(attemptedItemInfo.getStartId(), trackId));
                 LOG.warn("{} But adding trackID back to retry queue as some of"
                     + " the blocks couldn't find matching target nodes in"
                     + " previous SPS iteration.", msg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
index 41a3a6c..788a98b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
@@ -29,12 +29,15 @@ import java.util.Map;
 import java.util.Queue;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
 import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A Class to track the block collection IDs (Inode's ID) for which physical
  * storage movement needed as per the Namespace and StorageReports from DN.
@@ -53,11 +56,11 @@ public class BlockStorageMovementNeeded {
       new LinkedList<ItemInfo>();
 
   /**
-   * Map of rootId and number of child's. Number of child's indicate the number
-   * of files pending to satisfy the policy.
+   * Map of startId and number of child's. Number of child's indicate the
+   * number of files pending to satisfy the policy.
    */
-  private final Map<Long, Integer> pendingWorkForDirectory =
-      new HashMap<Long, Integer>();
+  private final Map<Long, DirPendingWorkInfo> pendingWorkForDirectory =
+      new HashMap<Long, DirPendingWorkInfo>();
 
   private final Namesystem namesystem;
 
@@ -66,12 +69,15 @@ public class BlockStorageMovementNeeded {
 
   private final StoragePolicySatisfier sps;
 
-  private Daemon fileInodeIdCollector;
+  private Daemon inodeIdCollector;
+
+  private final int maxQueuedItem;
 
   public BlockStorageMovementNeeded(Namesystem namesystem,
-      StoragePolicySatisfier sps) {
+      StoragePolicySatisfier sps, int queueLimit) {
     this.namesystem = namesystem;
     this.sps = sps;
+    this.maxQueuedItem = queueLimit;
   }
 
   /**
@@ -88,15 +94,24 @@ public class BlockStorageMovementNeeded {
   /**
    * Add the itemInfo to tracking list for which storage movement
    * expected if necessary.
-   * @param rootId
-   *            - root inode id
+   * @param startId
+   *            - start id
    * @param itemInfoList
    *            - List of child in the directory
    */
-  private synchronized void addAll(Long rootId,
-      List<ItemInfo> itemInfoList) {
+  @VisibleForTesting
+  public synchronized void addAll(long startId,
+      List<ItemInfo> itemInfoList, boolean scanCompleted) {
     storageMovementNeeded.addAll(itemInfoList);
-    pendingWorkForDirectory.put(rootId, itemInfoList.size());
+    DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
+    if (pendingWork == null) {
+      pendingWork = new DirPendingWorkInfo();
+      pendingWorkForDirectory.put(startId, pendingWork);
+    }
+    pendingWork.addPendingWorkCount(itemInfoList.size());
+    if (scanCompleted) {
+      pendingWork.markScanCompleted();
+    }
   }
 
   /**
@@ -118,6 +133,25 @@ public class BlockStorageMovementNeeded {
     }
   }
 
+  /**
+   * Returns queue remaining capacity.
+   */
+  public synchronized int remainingCapacity() {
+    int size = storageMovementNeeded.size();
+    if (size >= maxQueuedItem) {
+      return 0;
+    } else {
+      return (maxQueuedItem - size);
+    }
+  }
+
+  /**
+   * Returns queue size.
+   */
+  public synchronized int size() {
+    return storageMovementNeeded.size();
+  }
+
   public synchronized void clearAll() {
     spsDirsToBeTraveresed.clear();
     storageMovementNeeded.clear();
@@ -131,20 +165,20 @@ public class BlockStorageMovementNeeded {
   public synchronized void removeItemTrackInfo(ItemInfo trackInfo)
       throws IOException {
     if (trackInfo.isDir()) {
-      // If track is part of some root then reduce the pending directory work
-      // count.
-      long rootId = trackInfo.getRootId();
-      INode inode = namesystem.getFSDirectory().getInode(rootId);
+      // If track is part of some start inode then reduce the pending
+      // directory work count.
+      long startId = trackInfo.getStartId();
+      INode inode = namesystem.getFSDirectory().getInode(startId);
       if (inode == null) {
         // directory deleted just remove it.
-        this.pendingWorkForDirectory.remove(rootId);
+        this.pendingWorkForDirectory.remove(startId);
       } else {
-        if (pendingWorkForDirectory.get(rootId) != null) {
-          Integer pendingWork = pendingWorkForDirectory.get(rootId) - 1;
-          pendingWorkForDirectory.put(rootId, pendingWork);
-          if (pendingWork <= 0) {
-            namesystem.removeXattr(rootId, XATTR_SATISFY_STORAGE_POLICY);
-            pendingWorkForDirectory.remove(rootId);
+        DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
+        if (pendingWork != null) {
+          pendingWork.decrementPendingWorkCount();
+          if (pendingWork.isDirWorkDone()) {
+            namesystem.removeXattr(startId, XATTR_SATISFY_STORAGE_POLICY);
+            pendingWorkForDirectory.remove(startId);
           }
         }
       }
@@ -161,7 +195,7 @@ public class BlockStorageMovementNeeded {
     Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
     while (iterator.hasNext()) {
       ItemInfo next = iterator.next();
-      if (next.getRootId() == trackId) {
+      if (next.getStartId() == trackId) {
         iterator.remove();
       }
     }
@@ -208,7 +242,17 @@ public class BlockStorageMovementNeeded {
    * Take dir tack ID from the spsDirsToBeTraveresed queue and collect child
    * ID's to process for satisfy the policy.
    */
-  private class FileInodeIdCollector implements Runnable {
+  private class StorageMovementPendingInodeIdCollector extends FSTreeTraverser
+      implements Runnable {
+
+    private int remainingCapacity = 0;
+
+    private List<ItemInfo> currentBatch = new ArrayList<>(maxQueuedItem);
+
+    StorageMovementPendingInodeIdCollector(FSDirectory dir) {
+      super(dir);
+    }
+
     @Override
     public void run() {
       LOG.info("Starting FileInodeIdCollector!.");
@@ -216,38 +260,36 @@ public class BlockStorageMovementNeeded {
         try {
           if (!namesystem.isInSafeMode()) {
             FSDirectory fsd = namesystem.getFSDirectory();
-            Long rootINodeId = spsDirsToBeTraveresed.poll();
-            if (rootINodeId == null) {
+            Long startINodeId = spsDirsToBeTraveresed.poll();
+            if (startINodeId == null) {
               // Waiting for SPS path
               synchronized (spsDirsToBeTraveresed) {
                 spsDirsToBeTraveresed.wait(5000);
               }
             } else {
-              INode rootInode = fsd.getInode(rootINodeId);
-              if (rootInode != null) {
-                // TODO : HDFS-12291
-                // 1. Implement an efficient recursive directory iteration
-                // mechanism and satisfies storage policy for all the files
-                // under the given directory.
-                // 2. Process files in batches,so datanodes workload can be
-                // handled.
-                List<ItemInfo> itemInfoList =
-                    new ArrayList<>();
-                for (INode childInode : rootInode.asDirectory()
-                    .getChildrenList(Snapshot.CURRENT_STATE_ID)) {
-                  if (childInode.isFile()
-                      && childInode.asFile().numBlocks() != 0) {
-                    itemInfoList.add(
-                        new ItemInfo(rootINodeId, childInode.getId()));
-                  }
+              INode startInode = fsd.getInode(startINodeId);
+              if (startInode != null) {
+                try {
+                  remainingCapacity = remainingCapacity();
+                  readLock();
+                  traverseDir(startInode.asDirectory(), startINodeId,
+                      HdfsFileStatus.EMPTY_NAME,
+                      new SPSTraverseInfo(startINodeId));
+                } finally {
+                  readUnlock();
                 }
-                if (itemInfoList.isEmpty()) {
-                  // satisfy track info is empty, so remove the xAttr from the
-                  // directory
-                  namesystem.removeXattr(rootINodeId,
+                // Mark startInode traverse is done
+                addAll(startInode.getId(), currentBatch, true);
+                currentBatch.clear();
+
+                // check if directory was empty and no child added to queue
+                DirPendingWorkInfo dirPendingWorkInfo =
+                    pendingWorkForDirectory.get(startInode.getId());
+                if (dirPendingWorkInfo.isDirWorkDone()) {
+                  namesystem.removeXattr(startInode.getId(),
                       XATTR_SATISFY_STORAGE_POLICY);
+                  pendingWorkForDirectory.remove(startInode.getId());
                 }
-                addAll(rootINodeId, itemInfoList);
               }
             }
           }
@@ -256,17 +298,140 @@ public class BlockStorageMovementNeeded {
         }
       }
     }
+
+    @Override
+    protected void checkPauseForTesting() throws InterruptedException {
+      // TODO implement if needed
+    }
+
+    @Override
+    protected boolean processFileInode(INode inode, TraverseInfo traverseInfo)
+        throws IOException, InterruptedException {
+      assert getFSDirectory().hasReadLock();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Processing {} for statisy the policy",
+            inode.getFullPathName());
+      }
+      if (!inode.isFile()) {
+        return false;
+      }
+      if (inode.isFile() && inode.asFile().numBlocks() != 0) {
+        currentBatch.add(new ItemInfo(
+            ((SPSTraverseInfo) traverseInfo).getStartId(), inode.getId()));
+        remainingCapacity--;
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean canSubmitCurrentBatch() {
+      return remainingCapacity <= 0;
+    }
+
+    @Override
+    protected void checkINodeReady(long startId) throws IOException {
+      FSNamesystem fsn = ((FSNamesystem) namesystem);
+      fsn.checkNameNodeSafeMode("NN is in safe mode,"
+          + "cannot satisfy the policy.");
+      // SPS work should be cancelled when NN goes to standby. Just
+      // double checking for sanity.
+      fsn.checkOperation(NameNode.OperationCategory.WRITE);
+    }
+
+    @Override
+    protected void submitCurrentBatch(long startId)
+        throws IOException, InterruptedException {
+      // Add current child's to queue
+      addAll(startId, currentBatch, false);
+      currentBatch.clear();
+    }
+
+    @Override
+    protected void throttle() throws InterruptedException {
+      assert !getFSDirectory().hasReadLock();
+      assert !namesystem.hasReadLock();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("StorageMovementNeeded queue remaining capacity is zero,"
+            + " waiting for some free slots.");
+      }
+      remainingCapacity = remainingCapacity();
+      // wait for queue to be free
+      while (remainingCapacity <= 0) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Waiting for storageMovementNeeded queue to be free!");
+        }
+        Thread.sleep(5000);
+        remainingCapacity = remainingCapacity();
+      }
+    }
+
+    @Override
+    protected boolean canTraverseDir(INode inode) throws IOException {
+      return true;
+    }
   }
 
-  public void start() {
-    fileInodeIdCollector = new Daemon(new FileInodeIdCollector());
-    fileInodeIdCollector.setName("FileInodeIdCollector");
-    fileInodeIdCollector.start();
+  /**
+   * Info for directory recursive scan.
+   */
+  public static class DirPendingWorkInfo {
+
+    private int pendingWorkCount = 0;
+    private boolean fullyScanned = false;
+
+    /**
+     * Increment the pending work count for directory.
+     */
+    public synchronized void addPendingWorkCount(int count) {
+      this.pendingWorkCount = this.pendingWorkCount + count;
+    }
+
+    /**
+     * Decrement the pending work count for directory one track info is
+     * completed.
+     */
+    public synchronized void decrementPendingWorkCount() {
+      this.pendingWorkCount--;
+    }
+
+    /**
+     * Return true if all the pending work is done and directory fully
+     * scanned, otherwise false.
+     */
+    public synchronized boolean isDirWorkDone() {
+      return (pendingWorkCount <= 0 && fullyScanned);
+    }
+
+    /**
+     * Mark directory scan is completed.
+     */
+    public synchronized void markScanCompleted() {
+      this.fullyScanned = true;
+    }
   }
 
-  public void stop() {
-    if (fileInodeIdCollector != null) {
-      fileInodeIdCollector.interrupt();
+  public void init() {
+    inodeIdCollector = new Daemon(new StorageMovementPendingInodeIdCollector(
+        namesystem.getFSDirectory()));
+    inodeIdCollector.setName("FileInodeIdCollector");
+    inodeIdCollector.start();
+  }
+
+  public void close() {
+    if (inodeIdCollector != null) {
+      inodeIdCollector.interrupt();
+    }
+  }
+
+  class SPSTraverseInfo extends TraverseInfo {
+    private long startId;
+
+    SPSTraverseInfo(long startId) {
+      this.startId = startId;
+    }
+
+    public long getStartId() {
+      return startId;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeTraverser.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeTraverser.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeTraverser.java
new file mode 100644
index 0000000..acc23e5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeTraverser.java
@@ -0,0 +1,313 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * FSTreeTraverser traverse directory recursively and process files
+ * in batches.
+ */
+@InterfaceAudience.Private
+public abstract class FSTreeTraverser {
+
+  public static final Logger LOG = LoggerFactory
+      .getLogger(FSTreeTraverser.class);
+
+  private FSDirectory dir;
+
+  public FSTreeTraverser(FSDirectory dir) {
+    this.dir = dir;
+  }
+
+  public FSDirectory getFSDirectory() {
+    return dir;
+  }
+
+  /**
+   * Iterate through all files directly inside parent, and recurse down
+   * directories. The listing is done in batch, and can optionally start after
+   * a position. The iteration of the inode tree is done in a depth-first
+   * fashion. But instead of holding all {@link INodeDirectory}'s in memory
+   * on the fly, only the path components to the current inode is held. This
+   * is to reduce memory consumption.
+   *
+   * @param parent
+   *          The inode id of parent directory
+   * @param startId
+   *          Id of the start inode.
+   * @param startAfter
+   *          Full path of a file the traverse should start after.
+   * @param traverseInfo
+   *          info which may required for processing the child's.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  protected void traverseDir(final INodeDirectory parent, final long startId,
+      byte[] startAfter, final TraverseInfo traverseInfo)
+      throws IOException, InterruptedException {
+    List<byte[]> startAfters = new ArrayList<>();
+    if (parent == null) {
+      return;
+    }
+    INode curr = parent;
+    // construct startAfters all the way up to the zone inode.
+    startAfters.add(startAfter);
+    while (curr.getId() != startId) {
+      startAfters.add(0, curr.getLocalNameBytes());
+      curr = curr.getParent();
+    }
+    curr = traverseDirInt(startId, parent, startAfters, traverseInfo);
+    while (!startAfters.isEmpty()) {
+      if (curr == null) {
+        // lock was reacquired, re-resolve path.
+        curr = resolvePaths(startId, startAfters);
+      }
+      curr = traverseDirInt(startId, curr, startAfters, traverseInfo);
+    }
+  }
+
+  /**
+   * Iterates the parent directory, and add direct children files to current
+   * batch. If batch size meets configured threshold, current batch will be
+   * submitted for the processing.
+   * <p>
+   * Locks could be released and reacquired when a batch submission is
+   * finished.
+   *
+   * @param startId
+   *          Id of the start inode.
+   * @return The inode which was just processed, if lock is held in the entire
+   *         process. Null if lock is released.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  protected INode traverseDirInt(final long startId, INode curr,
+      List<byte[]> startAfters, final TraverseInfo traverseInfo)
+      throws IOException, InterruptedException {
+    assert dir.hasReadLock();
+    assert dir.getFSNamesystem().hasReadLock();
+    Preconditions.checkNotNull(curr, "Current inode can't be null");
+    checkINodeReady(startId);
+    final INodeDirectory parent = curr.isDirectory() ? curr.asDirectory()
+        : curr.getParent();
+    ReadOnlyList<INode> children = parent
+        .getChildrenList(Snapshot.CURRENT_STATE_ID);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Traversing directory {}", parent.getFullPathName());
+    }
+
+    final byte[] startAfter = startAfters.get(startAfters.size() - 1);
+    boolean lockReleased = false;
+    for (int i = INodeDirectory.nextChild(children, startAfter); i < children
+        .size(); ++i) {
+      final INode inode = children.get(i);
+      if (!processFileInode(inode, traverseInfo)) {
+        // inode wasn't processes. Recurse down if it's a dir,
+        // skip otherwise.
+        if (!inode.isDirectory()) {
+          continue;
+        }
+
+        if (!canTraverseDir(inode)) {
+          continue;
+        }
+        // add 1 level to the depth-first search.
+        curr = inode;
+        if (!startAfters.isEmpty()) {
+          startAfters.remove(startAfters.size() - 1);
+          startAfters.add(curr.getLocalNameBytes());
+        }
+        startAfters.add(HdfsFileStatus.EMPTY_NAME);
+        return lockReleased ? null : curr;
+      }
+      if (canSubmitCurrentBatch()) {
+        final byte[] currentStartAfter = inode.getLocalNameBytes();
+        final String parentPath = parent.getFullPathName();
+        lockReleased = true;
+        readUnlock();
+        submitCurrentBatch(startId);
+        try {
+          throttle();
+          checkPauseForTesting();
+        } finally {
+          readLock();
+        }
+        checkINodeReady(startId);
+
+        // Things could have changed when the lock was released.
+        // Re-resolve the parent inode.
+        FSPermissionChecker pc = dir.getPermissionChecker();
+        INode newParent = dir
+            .resolvePath(pc, parentPath, FSDirectory.DirOp.READ)
+            .getLastINode();
+        if (newParent == null || !newParent.equals(parent)) {
+          // parent dir is deleted or recreated. We're done.
+          return null;
+        }
+        children = parent.getChildrenList(Snapshot.CURRENT_STATE_ID);
+        // -1 to counter the ++ on the for loop
+        i = INodeDirectory.nextChild(children, currentStartAfter) - 1;
+      }
+    }
+    // Successfully finished this dir, adjust pointers to 1 level up, and
+    // startAfter this dir.
+    startAfters.remove(startAfters.size() - 1);
+    if (!startAfters.isEmpty()) {
+      startAfters.remove(startAfters.size() - 1);
+      startAfters.add(curr.getLocalNameBytes());
+    }
+    curr = curr.getParent();
+    return lockReleased ? null : curr;
+  }
+
+  /**
+   * Resolve the cursor of traverse to an inode.
+   * <p>
+   * The parent of the lowest level startAfter is returned. If somewhere in the
+   * middle of startAfters changed, the parent of the lowest unchanged level is
+   * returned.
+   *
+   * @param startId
+   *          Id of the start inode.
+   * @param startAfters
+   *          the cursor, represented by a list of path bytes.
+   * @return the parent inode corresponding to the startAfters, or null if the
+   *         furthest parent is deleted.
+   */
+  private INode resolvePaths(final long startId, List<byte[]> startAfters)
+      throws IOException {
+    // If the readlock was reacquired, we need to resolve the paths again
+    // in case things have changed. If our cursor file/dir is changed,
+    // continue from the next one.
+    INode zoneNode = dir.getInode(startId);
+    if (zoneNode == null) {
+      throw new FileNotFoundException("Zone " + startId + " is deleted.");
+    }
+    INodeDirectory parent = zoneNode.asDirectory();
+    for (int i = 0; i < startAfters.size(); ++i) {
+      if (i == startAfters.size() - 1) {
+        // last startAfter does not need to be resolved, since search for
+        // nextChild will cover that automatically.
+        break;
+      }
+      INode curr = parent.getChild(startAfters.get(i),
+          Snapshot.CURRENT_STATE_ID);
+      if (curr == null) {
+        // inode at this level has changed. Update startAfters to point to
+        // the next dir at the parent level (and dropping any startAfters
+        // at lower levels).
+        for (; i < startAfters.size(); ++i) {
+          startAfters.remove(startAfters.size() - 1);
+        }
+        break;
+      }
+      parent = curr.asDirectory();
+    }
+    return parent;
+  }
+
+  protected void readLock() {
+    dir.getFSNamesystem().readLock();
+    dir.readLock();
+  }
+
+  protected void readUnlock() {
+    dir.readUnlock();
+    dir.getFSNamesystem().readUnlock("FSTreeTraverser");
+  }
+
+
+  protected abstract void checkPauseForTesting() throws InterruptedException;
+
+  /**
+   * Process an Inode. Add to current batch if it's a file, no-op otherwise.
+   *
+   * @param inode
+   *          the inode
+   * @return true if inode is added to currentBatch and should be process for
+   *         next operation. false otherwise: could be inode is not a file.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  protected abstract boolean processFileInode(INode inode,
+      TraverseInfo traverseInfo) throws IOException, InterruptedException;
+
+  /**
+   * Check whether current batch can be submitted for the processing.
+   *
+   * @return true if batch size meets meet the condition, otherwise false.
+   */
+  protected abstract boolean canSubmitCurrentBatch();
+
+  /**
+   * Check whether inode is ready for traverse. Throws IOE if it's not.
+   *
+   * @param startId
+   *          Id of the start inode.
+   * @throws IOException
+   */
+  protected abstract void checkINodeReady(long startId) throws IOException;
+
+  /**
+   * Submit the current batch for processing.
+   *
+   * @param startId
+   *          Id of the start inode.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  protected abstract void submitCurrentBatch(long startId)
+      throws IOException, InterruptedException;
+
+  /**
+   * Throttles the FSTreeTraverser.
+   *
+   * @throws InterruptedException
+   */
+  protected abstract void throttle() throws InterruptedException;
+
+  /**
+   * Check whether dir is traversable or not.
+   *
+   * @param inode
+   *          Dir inode
+   * @return true if dir is traversable otherwise false.
+   * @throws IOException
+   */
+  protected abstract boolean canTraverseDir(INode inode) throws IOException;
+
+  /**
+   * Class will represent the additional info required for traverse.
+   */
+  public static class TraverseInfo {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
index 01c2038..9b00519 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
@@ -30,18 +31,16 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.ReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus.State;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
 import org.apache.hadoop.hdfs.server.namenode.ReencryptionUpdater.FileEdekInfo;
 import org.apache.hadoop.hdfs.server.namenode.ReencryptionUpdater.ReencryptionTask;
 import org.apache.hadoop.hdfs.server.namenode.ReencryptionUpdater.ZoneSubmissionTracker;
-import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StopWatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.GeneralSecurityException;
 import java.util.ArrayList;
@@ -118,6 +117,8 @@ public class ReencryptionHandler implements Runnable {
   // be single-threaded, see class javadoc for more details.
   private ReencryptionBatch currentBatch;
 
+  private ReencryptionPendingInodeIdCollector traverser;
+
   private final ReencryptionUpdater reencryptionUpdater;
   private ExecutorService updaterExecutor;
 
@@ -186,16 +187,6 @@ public class ReencryptionHandler implements Runnable {
     reencryptionUpdater.pauseForTestingAfterNthCheckpoint(zoneId, count);
   }
 
-  private synchronized void checkPauseForTesting() throws InterruptedException {
-    assert !dir.hasReadLock();
-    assert !dir.getFSNamesystem().hasReadLock();
-    while (shouldPauseForTesting) {
-      LOG.info("Sleeping in the re-encrypt handler for unit test.");
-      wait();
-      LOG.info("Continuing re-encrypt handler after pausing.");
-    }
-  }
-
   ReencryptionHandler(final EncryptionZoneManager ezMgr,
       final Configuration conf) {
     this.ezManager = ezMgr;
@@ -256,6 +247,7 @@ public class ReencryptionHandler implements Runnable {
     reencryptionUpdater =
         new ReencryptionUpdater(dir, batchService, this, conf);
     currentBatch = new ReencryptionBatch(reencryptBatchSize);
+    traverser = new ReencryptionPendingInodeIdCollector(dir, this);
   }
 
   ReencryptionStatus getReencryptionStatus() {
@@ -339,7 +331,7 @@ public class ReencryptionHandler implements Runnable {
         synchronized (this) {
           wait(interval);
         }
-        checkPauseForTesting();
+        traverser.checkPauseForTesting();
       } catch (InterruptedException ie) {
         LOG.info("Re-encrypt handler interrupted. Exiting");
         Thread.currentThread().interrupt();
@@ -397,7 +389,7 @@ public class ReencryptionHandler implements Runnable {
     final INode zoneNode;
     final ZoneReencryptionStatus zs;
 
-    readLock();
+    traverser.readLock();
     try {
       zoneNode = dir.getInode(zoneId);
       // start re-encrypting the zone from the beginning
@@ -419,18 +411,19 @@ public class ReencryptionHandler implements Runnable {
           zoneId);
       if (zs.getLastCheckpointFile() == null) {
         // new re-encryption
-        reencryptDir(zoneNode.asDirectory(), zoneId, HdfsFileStatus.EMPTY_NAME,
-            zs.getEzKeyVersionName());
+        traverser.traverseDir(zoneNode.asDirectory(), zoneId,
+            HdfsFileStatus.EMPTY_NAME,
+            new ZoneTraverseInfo(zs.getEzKeyVersionName()));
       } else {
         // resuming from a past re-encryption
         restoreFromLastProcessedFile(zoneId, zs);
       }
       // save the last batch and mark complete
-      submitCurrentBatch(zoneId);
+      traverser.submitCurrentBatch(zoneId);
       LOG.info("Submission completed of zone {} for re-encryption.", zoneId);
       reencryptionUpdater.markZoneSubmissionDone(zoneId);
     } finally {
-      readUnlock();
+      traverser.readUnlock();
     }
   }
 
@@ -479,131 +472,8 @@ public class ReencryptionHandler implements Runnable {
         dir.getINodesInPath(zs.getLastCheckpointFile(), FSDirectory.DirOp.READ);
     parent = lpfIIP.getLastINode().getParent();
     startAfter = lpfIIP.getLastINode().getLocalNameBytes();
-    reencryptDir(parent, zoneId, startAfter, zs.getEzKeyVersionName());
-  }
-
-  /**
-   * Iterate through all files directly inside parent, and recurse down
-   * directories. The listing is done in batch, and can optionally start after
-   * a position.
-   * <p>
-   * Each batch is then send to the threadpool, where KMS will be contacted and
-   * edek re-encrypted. {@link ReencryptionUpdater} handles the tasks completed
-   * from the threadpool.
-   * <p>
-   * The iteration of the inode tree is done in a depth-first fashion. But
-   * instead of holding all INodeDirectory's in memory on the fly, only the
-   * path components to the current inode is held. This is to reduce memory
-   * consumption.
-   *
-   * @param parent     The inode id of parent directory
-   * @param zoneId     Id of the EZ inode
-   * @param startAfter Full path of a file the re-encrypt should start after.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  private void reencryptDir(final INodeDirectory parent, final long zoneId,
-      byte[] startAfter, final String ezKeyVerName)
-      throws IOException, InterruptedException {
-    List<byte[]> startAfters = new ArrayList<>();
-    if (parent == null) {
-      return;
-    }
-    INode curr = parent;
-    // construct startAfters all the way up to the zone inode.
-    startAfters.add(startAfter);
-    while (curr.getId() != zoneId) {
-      startAfters.add(0, curr.getLocalNameBytes());
-      curr = curr.getParent();
-    }
-    curr = reencryptDirInt(zoneId, parent, startAfters, ezKeyVerName);
-    while (!startAfters.isEmpty()) {
-      if (curr == null) {
-        // lock was reacquired, re-resolve path.
-        curr = resolvePaths(zoneId, startAfters);
-      }
-      curr = reencryptDirInt(zoneId, curr, startAfters, ezKeyVerName);
-    }
-  }
-
-  /**
-   * Resolve the cursor of re-encryption to an inode.
-   * <p>
-   * The parent of the lowest level startAfter is returned. If somewhere in the
-   * middle of startAfters changed, the parent of the lowest unchanged level is
-   * returned.
-   *
-   * @param zoneId      Id of the EZ inode.
-   * @param startAfters the cursor, represented by a list of path bytes.
-   * @return the parent inode corresponding to the startAfters, or null if
-   * the EZ node (furthest parent) is deleted.
-   */
-  private INode resolvePaths(final long zoneId, List<byte[]> startAfters)
-      throws IOException {
-    // If the readlock was reacquired, we need to resolve the paths again
-    // in case things have changed. If our cursor file/dir is changed,
-    // continue from the next one.
-    INode zoneNode = dir.getInode(zoneId);
-    if (zoneNode == null) {
-      throw new FileNotFoundException("Zone " + zoneId + " is deleted.");
-    }
-    INodeDirectory parent = zoneNode.asDirectory();
-    for (int i = 0; i < startAfters.size(); ++i) {
-      if (i == startAfters.size() - 1) {
-        // last startAfter does not need to be resolved, since search for
-        // nextChild will cover that automatically.
-        break;
-      }
-      INode curr =
-          parent.getChild(startAfters.get(i), Snapshot.CURRENT_STATE_ID);
-      if (curr == null) {
-        // inode at this level has changed. Update startAfters to point to
-        // the next dir at the parent level (and dropping any startAfters
-        // at lower levels).
-        for (; i < startAfters.size(); ++i) {
-          startAfters.remove(startAfters.size() - 1);
-        }
-        break;
-      }
-      parent = curr.asDirectory();
-    }
-    return parent;
-  }
-
-  /**
-   * Submit the current batch to the thread pool.
-   *
-   * @param zoneId Id of the EZ INode
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  private void submitCurrentBatch(final long zoneId)
-      throws IOException, InterruptedException {
-    assert dir.hasReadLock();
-    if (currentBatch.isEmpty()) {
-      return;
-    }
-    ZoneSubmissionTracker zst;
-    synchronized (this) {
-      zst = submissions.get(zoneId);
-      if (zst == null) {
-        zst = new ZoneSubmissionTracker();
-        submissions.put(zoneId, zst);
-      }
-    }
-    Future future = batchService
-        .submit(new EDEKReencryptCallable(zoneId, currentBatch, this));
-    zst.addTask(future);
-    LOG.info("Submitted batch (start:{}, size:{}) of zone {} to re-encrypt.",
-        currentBatch.getFirstFilePath(), currentBatch.size(), zoneId);
-    currentBatch = new ReencryptionBatch(reencryptBatchSize);
-    // flip the pause flag if this is nth submission.
-    // The actual pause need to happen outside of the lock.
-    if (pauseAfterNthSubmission > 0) {
-      if (--pauseAfterNthSubmission == 0) {
-        shouldPauseForTesting = true;
-      }
-    }
+    traverser.traverseDir(parent, zoneId, startAfter,
+        new ZoneTraverseInfo(zs.getEzKeyVersionName()));
   }
 
   final class ReencryptionBatch {
@@ -711,256 +581,270 @@ public class ReencryptionHandler implements Runnable {
     }
   }
 
+
   /**
-   * Iterates the parent directory, and add direct children files to
-   * current batch. If batch size meets configured threshold, a Callable
-   * is created and sent to the thread pool, which will communicate to the KMS
-   * to get new edeks.
-   * <p>
-   * Locks could be released and reacquired when a Callable is created.
-   *
-   * @param zoneId Id of the EZ INode
-   * @return The inode which was just processed, if lock is held in the entire
-   * process. Null if lock is released.
-   * @throws IOException
-   * @throws InterruptedException
+   * Called when a new zone is submitted for re-encryption. This will interrupt
+   * the background thread if it's waiting for the next
+   * DFS_NAMENODE_REENCRYPT_SLEEP_INTERVAL_KEY.
    */
-  private INode reencryptDirInt(final long zoneId, INode curr,
-      List<byte[]> startAfters, final String ezKeyVerName)
-      throws IOException, InterruptedException {
-    assert dir.hasReadLock();
-    assert dir.getFSNamesystem().hasReadLock();
-    Preconditions.checkNotNull(curr, "Current inode can't be null");
-    checkZoneReady(zoneId);
-    final INodeDirectory parent =
-        curr.isDirectory() ? curr.asDirectory() : curr.getParent();
-    ReadOnlyList<INode> children =
-        parent.getChildrenList(Snapshot.CURRENT_STATE_ID);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Re-encrypting directory {}", parent.getFullPathName());
-    }
-
-    final byte[] startAfter = startAfters.get(startAfters.size() - 1);
-    boolean lockReleased = false;
-    for (int i = INodeDirectory.nextChild(children, startAfter);
-         i < children.size(); ++i) {
-      final INode inode = children.get(i);
-      if (!reencryptINode(inode, ezKeyVerName)) {
-        // inode wasn't added for re-encryption. Recurse down if it's a dir,
-        // skip otherwise.
-        if (!inode.isDirectory()) {
-          continue;
-        }
-        if (ezManager.isEncryptionZoneRoot(inode, inode.getFullPathName())) {
-          // nested EZ, ignore.
-          LOG.info("{}({}) is a nested EZ, skipping for re-encryption",
-              inode.getFullPathName(), inode.getId());
-          continue;
+  synchronized void notifyNewSubmission() {
+    LOG.debug("Notifying handler for new re-encryption command.");
+    this.notify();
+  }
+
+  public ReencryptionPendingInodeIdCollector getTraverser() {
+    return traverser;
+  }
+
+  /**
+   * ReencryptionPendingInodeIdCollector which throttle based on configured
+   * throttle ratio.
+   */
+  class ReencryptionPendingInodeIdCollector extends FSTreeTraverser {
+
+    private ReencryptionHandler reencryptionHandler;
+
+    ReencryptionPendingInodeIdCollector(FSDirectory dir,
+        ReencryptionHandler rHandler) {
+      super(dir);
+      this.reencryptionHandler = rHandler;
+    }
+
+    @Override
+    protected void checkPauseForTesting()
+        throws InterruptedException {
+      assert !dir.hasReadLock();
+      assert !dir.getFSNamesystem().hasReadLock();
+      while (shouldPauseForTesting) {
+        LOG.info("Sleeping in the re-encrypt handler for unit test.");
+        synchronized (reencryptionHandler) {
+          reencryptionHandler.wait(30000);
         }
-        // add 1 level to the depth-first search.
-        curr = inode;
-        if (!startAfters.isEmpty()) {
-          startAfters.remove(startAfters.size() - 1);
-          startAfters.add(curr.getLocalNameBytes());
+        LOG.info("Continuing re-encrypt handler after pausing.");
+      }
+    }
+
+    /**
+     * Process an Inode for re-encryption. Add to current batch if it's a file,
+     * no-op otherwise.
+     *
+     * @param inode
+     *          the inode
+     * @return true if inode is added to currentBatch and should be
+     *         re-encrypted. false otherwise: could be inode is not a file, or
+     *         inode's edek's key version is not changed.
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Override
+    public boolean processFileInode(INode inode, TraverseInfo traverseInfo)
+        throws IOException, InterruptedException {
+      assert dir.hasReadLock();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Processing {} for re-encryption", inode.getFullPathName());
+      }
+      if (!inode.isFile()) {
+        return false;
+      }
+      FileEncryptionInfo feInfo = FSDirEncryptionZoneOp.getFileEncryptionInfo(
+          dir, INodesInPath.fromINode(inode));
+      if (feInfo == null) {
+        LOG.warn("File {} skipped re-encryption because it is not encrypted! "
+            + "This is very likely a bug.", inode.getId());
+        return false;
+      }
+      if (traverseInfo instanceof ZoneTraverseInfo
+          && ((ZoneTraverseInfo) traverseInfo).getEzKeyVerName().equals(
+              feInfo.getEzKeyVersionName())) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("File {} skipped re-encryption because edek's key version"
+              + " name is not changed.", inode.getFullPathName());
         }
-        startAfters.add(HdfsFileStatus.EMPTY_NAME);
-        return lockReleased ? null : curr;
+        return false;
+      }
+      currentBatch.add(inode.asFile());
+      return true;
+    }
+
+    /**
+     * Check whether zone is ready for re-encryption. Throws IOE if it's not. 1.
+     * If EZ is deleted. 2. if the re-encryption is canceled. 3. If NN is not
+     * active or is in safe mode.
+     *
+     * @throws IOException
+     *           if zone does not exist / is cancelled, or if NN is not ready
+     *           for write.
+     */
+    @Override
+    protected void checkINodeReady(long zoneId) throws IOException {
+      final ZoneReencryptionStatus zs = getReencryptionStatus().getZoneStatus(
+          zoneId);
+      if (zs == null) {
+        throw new IOException("Zone " + zoneId + " status cannot be found.");
+      }
+      if (zs.isCanceled()) {
+        throw new IOException("Re-encryption is canceled for zone " + zoneId);
       }
-      if (currentBatch.size() >= reencryptBatchSize) {
-        final byte[] currentStartAfter = inode.getLocalNameBytes();
-        final String parentPath = parent.getFullPathName();
-        submitCurrentBatch(zoneId);
-        lockReleased = true;
-        readUnlock();
-        try {
-          throttle();
-          checkPauseForTesting();
-        } finally {
-          readLock();
+      dir.getFSNamesystem().checkNameNodeSafeMode(
+          "NN is in safe mode, cannot re-encrypt.");
+      // re-encryption should be cancelled when NN goes to standby. Just
+      // double checking for sanity.
+      dir.getFSNamesystem().checkOperation(NameNode.OperationCategory.WRITE);
+    }
+
+    /**
+     * Submit the current batch to the thread pool.
+     *
+     * @param zoneId
+     *          Id of the EZ INode
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Override
+    protected void submitCurrentBatch(final long zoneId) throws IOException,
+        InterruptedException {
+      if (currentBatch.isEmpty()) {
+        return;
+      }
+      ZoneSubmissionTracker zst;
+      synchronized (ReencryptionHandler.this) {
+        zst = submissions.get(zoneId);
+        if (zst == null) {
+          zst = new ZoneSubmissionTracker();
+          submissions.put(zoneId, zst);
         }
-        checkZoneReady(zoneId);
-
-        // Things could have changed when the lock was released.
-        // Re-resolve the parent inode.
-        FSPermissionChecker pc = dir.getPermissionChecker();
-        INode newParent =
-            dir.resolvePath(pc, parentPath, FSDirectory.DirOp.READ)
-                .getLastINode();
-        if (newParent == null || !newParent.equals(parent)) {
-          // parent dir is deleted or recreated. We're done.
-          return null;
+      }
+      Future future = batchService.submit(new EDEKReencryptCallable(zoneId,
+          currentBatch, reencryptionHandler));
+      zst.addTask(future);
+      LOG.info("Submitted batch (start:{}, size:{}) of zone {} to re-encrypt.",
+          currentBatch.getFirstFilePath(), currentBatch.size(), zoneId);
+      currentBatch = new ReencryptionBatch(reencryptBatchSize);
+      // flip the pause flag if this is nth submission.
+      // The actual pause need to happen outside of the lock.
+      if (pauseAfterNthSubmission > 0) {
+        if (--pauseAfterNthSubmission == 0) {
+          shouldPauseForTesting = true;
         }
-        children = parent.getChildrenList(Snapshot.CURRENT_STATE_ID);
-        // -1 to counter the ++ on the for loop
-        i = INodeDirectory.nextChild(children, currentStartAfter) - 1;
       }
     }
-    // Successfully finished this dir, adjust pointers to 1 level up, and
-    // startAfter this dir.
-    startAfters.remove(startAfters.size() - 1);
-    if (!startAfters.isEmpty()) {
-      startAfters.remove(startAfters.size() - 1);
-      startAfters.add(curr.getLocalNameBytes());
-    }
-    curr = curr.getParent();
-    return lockReleased ? null : curr;
-  }
 
-  private void readLock() {
-    dir.getFSNamesystem().readLock();
-    dir.readLock();
-    throttleTimerLocked.start();
-  }
+    /**
+     * Throttles the ReencryptionHandler in 3 aspects:
+     * 1. Prevents generating more Callables than the CPU could possibly
+     * handle.
+     * 2. Prevents generating more Callables than the ReencryptionUpdater
+     * can handle, under its own throttling.
+     * 3. Prevents contending FSN/FSD read locks. This is done based
+     * on the DFS_NAMENODE_REENCRYPT_THROTTLE_LIMIT_RATIO_KEY configuration.
+     * <p>
+     * Item 1 and 2 are to control NN heap usage.
+     *
+     * @throws InterruptedException
+     */
+    @VisibleForTesting
+    @Override
+    protected void throttle() throws InterruptedException {
+      assert !dir.hasReadLock();
+      assert !dir.getFSNamesystem().hasReadLock();
+      final int numCores = Runtime.getRuntime().availableProcessors();
+      if (taskQueue.size() >= numCores) {
+        LOG.debug("Re-encryption handler throttling because queue size {} is"
+            + "larger than number of cores {}", taskQueue.size(), numCores);
+        while (taskQueue.size() >= numCores) {
+          Thread.sleep(100);
+        }
+      }
 
-  private void readUnlock() {
-    dir.readUnlock();
-    dir.getFSNamesystem().readUnlock("reencryptHandler");
-    throttleTimerLocked.stop();
-  }
+      // 2. if tasks are piling up on the updater, don't create new callables
+      // until the queue size goes down.
+      final int maxTasksPiled = Runtime.getRuntime().availableProcessors() * 2;
+      int numTasks = numTasksSubmitted();
+      if (numTasks >= maxTasksPiled) {
+        LOG.debug("Re-encryption handler throttling because total tasks pending"
+            + " re-encryption updater is {}", numTasks);
+        while (numTasks >= maxTasksPiled) {
+          Thread.sleep(500);
+          numTasks = numTasksSubmitted();
+        }
+      }
 
-  /**
-   * Throttles the ReencryptionHandler in 3 aspects:
-   * 1. Prevents generating more Callables than the CPU could possibly handle.
-   * 2. Prevents generating more Callables than the ReencryptionUpdater can
-   *   handle, under its own throttling
-   * 3. Prevents contending FSN/FSD read locks. This is done based on the
-   *   DFS_NAMENODE_REENCRYPT_THROTTLE_LIMIT_RATIO_KEY configuration.
-   * <p>
-   * Item 1 and 2 are to control NN heap usage.
-   *
-   * @throws InterruptedException
-   */
-  @VisibleForTesting
-  void throttle() throws InterruptedException {
-    // 1.
-    final int numCores = Runtime.getRuntime().availableProcessors();
-    if (taskQueue.size() >= numCores) {
-      LOG.debug("Re-encryption handler throttling because queue size {} is"
-          + "larger than number of cores {}", taskQueue.size(), numCores);
-      while (taskQueue.size() >= numCores) {
-        Thread.sleep(100);
+      // 3.
+      if (throttleLimitHandlerRatio >= 1.0) {
+        return;
+      }
+      final long expect = (long) (throttleTimerAll.now(TimeUnit.MILLISECONDS)
+          * throttleLimitHandlerRatio);
+      final long actual = throttleTimerLocked.now(TimeUnit.MILLISECONDS);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Re-encryption handler throttling expect: {}, actual: {},"
+            + " throttleTimerAll:{}", expect, actual,
+            throttleTimerAll.now(TimeUnit.MILLISECONDS));
       }
+      if (expect - actual < 0) {
+        // in case throttleLimitHandlerRatio is very small, expect will be 0.
+        // so sleepMs should not be calculated from expect, to really meet the
+        // ratio. e.g. if ratio is 0.001, expect = 0 and actual = 1, sleepMs
+        // should be 1000 - throttleTimerAll.now()
+        final long sleepMs = (long) (actual / throttleLimitHandlerRatio)
+            - throttleTimerAll.now(TimeUnit.MILLISECONDS);
+        LOG.debug("Throttling re-encryption, sleeping for {} ms", sleepMs);
+        Thread.sleep(sleepMs);
+      }
+      throttleTimerAll.reset().start();
+      throttleTimerLocked.reset();
     }
 
-    // 2. if tasks are piling up on the updater, don't create new callables
-    // until the queue size goes down.
-    final int maxTasksPiled = Runtime.getRuntime().availableProcessors() * 2;
-    int numTasks = numTasksSubmitted();
-    if (numTasks >= maxTasksPiled) {
-      LOG.debug("Re-encryption handler throttling because total tasks pending"
-          + " re-encryption updater is {}", numTasks);
-      while (numTasks >= maxTasksPiled) {
-        Thread.sleep(500);
-        numTasks = numTasksSubmitted();
+    private int numTasksSubmitted() {
+      int ret = 0;
+      synchronized (ReencryptionHandler.this) {
+        for (ZoneSubmissionTracker zst : submissions.values()) {
+          ret += zst.getTasks().size();
+        }
       }
+      return ret;
     }
 
-    // 3.
-    if (throttleLimitHandlerRatio >= 1.0) {
-      return;
-    }
-    final long expect = (long) (throttleTimerAll.now(TimeUnit.MILLISECONDS)
-        * throttleLimitHandlerRatio);
-    final long actual = throttleTimerLocked.now(TimeUnit.MILLISECONDS);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Re-encryption handler throttling expect: {}, actual: {},"
-              + " throttleTimerAll:{}", expect, actual,
-          throttleTimerAll.now(TimeUnit.MILLISECONDS));
-    }
-    if (expect - actual < 0) {
-      // in case throttleLimitHandlerRatio is very small, expect will be 0.
-      // so sleepMs should not be calculated from expect, to really meet the
-      // ratio. e.g. if ratio is 0.001, expect = 0 and actual = 1, sleepMs
-      // should be 1000 - throttleTimerAll.now()
-      final long sleepMs =
-          (long) (actual / throttleLimitHandlerRatio) - throttleTimerAll
-              .now(TimeUnit.MILLISECONDS);
-      LOG.debug("Throttling re-encryption, sleeping for {} ms", sleepMs);
-      Thread.sleep(sleepMs);
+    @Override
+    public boolean canSubmitCurrentBatch() {
+      return currentBatch.size() >= reencryptBatchSize;
     }
-    throttleTimerAll.reset().start();
-    throttleTimerLocked.reset();
-  }
 
-  private synchronized int numTasksSubmitted() {
-    int ret = 0;
-    for (ZoneSubmissionTracker zst : submissions.values()) {
-      ret += zst.getTasks().size();
+    @Override
+    public boolean canTraverseDir(INode inode) throws IOException {
+      if (ezManager.isEncryptionZoneRoot(inode, inode.getFullPathName())) {
+        // nested EZ, ignore.
+        LOG.info("{}({}) is a nested EZ, skipping for re-encryption",
+            inode.getFullPathName(), inode.getId());
+        return false;
+      }
+      return true;
     }
-    return ret;
-  }
 
-  /**
-   * Process an Inode for re-encryption. Add to current batch if it's a file,
-   * no-op otherwise.
-   *
-   * @param inode the inode
-   * @return true if inode is added to currentBatch and should be re-encrypted.
-   * false otherwise: could be inode is not a file, or inode's edek's
-   * key version is not changed.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  private boolean reencryptINode(final INode inode, final String ezKeyVerName)
-      throws IOException, InterruptedException {
-    assert dir.hasReadLock();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Processing {} for re-encryption", inode.getFullPathName());
-    }
-    if (!inode.isFile()) {
-      return false;
-    }
-    FileEncryptionInfo feInfo = FSDirEncryptionZoneOp
-        .getFileEncryptionInfo(dir, INodesInPath.fromINode(inode));
-    if (feInfo == null) {
-      LOG.warn("File {} skipped re-encryption because it is not encrypted! "
-          + "This is very likely a bug.", inode.getId());
-      return false;
+    @Override
+    protected void readLock() {
+      dir.getFSNamesystem().readLock();
+      dir.readLock();
+      throttleTimerLocked.start();
     }
-    if (ezKeyVerName.equals(feInfo.getEzKeyVersionName())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("File {} skipped re-encryption because edek's key version"
-            + " name is not changed.", inode.getFullPathName());
-      }
-      return false;
+
+    @Override
+    protected void readUnlock() {
+      dir.readUnlock();
+      dir.getFSNamesystem().readUnlock("reencryptHandler");
+      throttleTimerLocked.stop();
     }
-    currentBatch.add(inode.asFile());
-    return true;
   }
 
-  /**
-   * Check whether zone is ready for re-encryption. Throws IOE if it's not.
-   * 1. If EZ is deleted.
-   * 2. if the re-encryption is canceled.
-   * 3. If NN is not active or is in safe mode.
-   *
-   * @throws IOException if zone does not exist / is cancelled, or if NN is not
-   *                     ready for write.
-   */
-  void checkZoneReady(final long zoneId)
-      throws RetriableException, SafeModeException, IOException {
-    final ZoneReencryptionStatus zs =
-        getReencryptionStatus().getZoneStatus(zoneId);
-    if (zs == null) {
-      throw new IOException("Zone " + zoneId + " status cannot be found.");
-    }
-    if (zs.isCanceled()) {
-      throw new IOException("Re-encryption is canceled for zone " + zoneId);
+  class ZoneTraverseInfo extends TraverseInfo {
+    private String ezKeyVerName;
+
+    ZoneTraverseInfo(String ezKeyVerName) {
+      this.ezKeyVerName = ezKeyVerName;
     }
-    dir.getFSNamesystem()
-        .checkNameNodeSafeMode("NN is in safe mode, cannot re-encrypt.");
-    // re-encryption should be cancelled when NN goes to standby. Just
-    // double checking for sanity.
-    dir.getFSNamesystem().checkOperation(NameNode.OperationCategory.WRITE);
-  }
 
-  /**
-   * Called when a new zone is submitted for re-encryption. This will interrupt
-   * the background thread if it's waiting for the next
-   * DFS_NAMENODE_REENCRYPT_SLEEP_INTERVAL_KEY.
-   */
-  synchronized void notifyNewSubmission() {
-    LOG.debug("Notifying handler for new re-encryption command.");
-    this.notify();
+    public String getEzKeyVerName() {
+      return ezKeyVerName;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
index 3b7badb..a5923a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
@@ -464,7 +464,7 @@ public final class ReencryptionUpdater implements Runnable {
     final String zonePath;
     dir.writeLock();
     try {
-      handler.checkZoneReady(task.zoneId);
+      handler.getTraverser().checkINodeReady(task.zoneId);
       final INode zoneNode = dir.getInode(task.zoneId);
       if (zoneNode == null) {
         // ez removed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 48d0598..a4372d5 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
@@ -77,7 +77,8 @@ public class StoragePolicySatisfier implements Runnable {
   private final BlockStorageMovementNeeded storageMovementNeeded;
   private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
   private volatile boolean isRunning = false;
-
+  private int spsWorkMultiplier;
+  private long blockCount = 0L;
   /**
    * Represents the collective analysis status for all blocks.
    */
@@ -106,7 +107,9 @@ public class StoragePolicySatisfier implements Runnable {
       final BlockManager blkManager, Configuration conf) {
     this.namesystem = namesystem;
     this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem,
-        this);
+        this, conf.getInt(
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
+            DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT));
     this.blockManager = blkManager;
     this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(
         conf.getLong(
@@ -117,6 +120,7 @@ public class StoragePolicySatisfier implements Runnable {
             DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
         storageMovementNeeded,
         this);
+    this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(conf);
   }
 
   /**
@@ -143,7 +147,7 @@ public class StoragePolicySatisfier implements Runnable {
     // Ensure that all the previously submitted block movements(if any) have to
     // be stopped in all datanodes.
     addDropSPSWorkCommandsToAllDNs();
-    storageMovementNeeded.start();
+    storageMovementNeeded.init();
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
@@ -164,7 +168,7 @@ public class StoragePolicySatisfier implements Runnable {
       return;
     }
 
-    storageMovementNeeded.stop();
+    storageMovementNeeded.close();
 
     storagePolicySatisfierThread.interrupt();
     this.storageMovementsMonitor.stop();
@@ -268,9 +272,13 @@ public class StoragePolicySatisfier implements Runnable {
             }
           }
         }
-        // TODO: We can think to make this as configurable later, how frequently
-        // we want to check block movements.
-        Thread.sleep(3000);
+        int numLiveDn = namesystem.getFSDirectory().getBlockManager()
+            .getDatanodeManager().getNumLiveDataNodes();
+        if (storageMovementNeeded.size() == 0
+            || blockCount > (numLiveDn * spsWorkMultiplier)) {
+          Thread.sleep(3000);
+          blockCount = 0L;
+        }
       } catch (Throwable t) {
         handleException(t);
       }
@@ -380,6 +388,11 @@ public class StoragePolicySatisfier implements Runnable {
 
     assignBlockMovingInfosToCoordinatorDn(blockCollection.getId(),
         blockMovingInfos, coordinatorNode);
+    int count = 0;
+    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
+      count = count + blkMovingInfo.getSources().length;
+    }
+    blockCount = blockCount + count;
     return status;
   }
 
@@ -840,7 +853,7 @@ public class StoragePolicySatisfier implements Runnable {
    *          - file inode/blockcollection id.
    */
   public void satisfyStoragePolicy(Long inodeId) {
-    //For file rootId and trackId is same
+    //For file startId and trackId is same
     storageMovementNeeded.add(new ItemInfo(inodeId, inodeId));
     if (LOG.isDebugEnabled()) {
       LOG.debug("Added track info for inode {} to block "
@@ -864,19 +877,19 @@ public class StoragePolicySatisfier implements Runnable {
    * policy.
    */
   public static class ItemInfo {
-    private long rootId;
+    private long startId;
     private long trackId;
 
-    public ItemInfo(long rootId, long trackId) {
-      this.rootId = rootId;
+    public ItemInfo(long startId, long trackId) {
+      this.startId = startId;
       this.trackId = trackId;
     }
 
     /**
-     * Return the root of the current track Id.
+     * Return the start inode id of the current track Id.
      */
-    public long getRootId() {
-      return rootId;
+    public long getStartId() {
+      return startId;
     }
 
     /**
@@ -890,7 +903,7 @@ public class StoragePolicySatisfier implements Runnable {
      * Returns true if the tracking path is a directory, false otherwise.
      */
     public boolean isDir() {
-      return (rootId != trackId);
+      return (startId != trackId);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 22d4bd8..ef0bf12 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
@@ -4358,6 +4358,29 @@
 </property>
 
 <property>
+  <name>dfs.storage.policy.satisfier.queue.limit</name>
+  <value>1000</value>
+  <description>
+    Storage policy satisfier queue size. This queue contains the currently
+    scheduled file's inode ID for statisfy the policy.
+    Default value is 1000.
+  </description>
+</property>
+
+<property>
+  <name>dfs.storage.policy.satisfier.work.multiplier.per.iteration</name>
+  <value>1</value>
+  <description>
+    *Note*: Advanced property. Change with caution.
+    This determines the total amount of block transfers to begin in
+    one iteration, for satisfy the policy. The actual number is obtained by
+    multiplying this multiplier with the total number of live nodes in the
+    cluster. The result number is the number of blocks to begin transfers
+    immediately. This number can be any positive, non-zero integer.
+  </description>
+</property>
+
+<property>
   <name>dfs.storage.policy.satisfier.recheck.timeout.millis</name>
   <value>300000</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 87817cf..da61842 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -110,7 +110,7 @@ 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)
 
-* **Note**: When user invokes `satisfyStoragePolicy()` API on a directory, SPS will consider the files which are immediate to that directory. Sub-directories won't be considered for satisfying the policy. Its user responsibility to call this API on directories recursively, to track all files under the sub tree.
+* **Note**: When user invokes `satisfyStoragePolicy()` API on a directory, SPS will scan all sub-directories and consider all the files for satisfy the policy..
 
 * HdfsAdmin API :
         `public void satisfyStoragePolicy(final Path path) throws IOException`
@@ -212,7 +212,6 @@ Get the storage policy of a file or a directory.
 ### Satisfy Storage Policy
 
 Schedule blocks to move based on file's/directory's current storage policy.
-Note: For directory case, it will consider immediate files under that directory and it will not consider sub directories recursively.
 
 * Command:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 55ebf9c..7918821 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
@@ -41,7 +41,7 @@ public class TestBlockStorageMovementAttemptedItems {
   public void setup() throws Exception {
     unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(
         Mockito.mock(Namesystem.class),
-        Mockito.mock(StoragePolicySatisfier.class));
+        Mockito.mock(StoragePolicySatisfier.class), 100);
     StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
     bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
         selfRetryTimeout, unsatisfiedStorageMovementFiles, sps);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/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 e7b9148..5bce296 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
@@ -191,7 +191,7 @@ public class TestPersistentStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           parentFileName, StorageType.ARCHIVE, 3, timeout, fs);
       DFSTestUtil.waitExpectedStorageType(
-          childFileName, StorageType.DEFAULT, 3, timeout, fs);
+          childFileName, StorageType.ARCHIVE, 3, timeout, fs);
 
     } finally {
       clusterShutdown();
@@ -232,7 +232,9 @@ public class TestPersistentStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           parentFileName, StorageType.ARCHIVE, 2, timeout, fs);
       DFSTestUtil.waitExpectedStorageType(
-          childFileName, StorageType.DEFAULT, 3, timeout, fs);
+          childFileName, StorageType.DISK, 1, timeout, fs);
+      DFSTestUtil.waitExpectedStorageType(
+          childFileName, StorageType.ARCHIVE, 2, timeout, fs);
     } finally {
       clusterShutdown();
     }
@@ -269,7 +271,7 @@ public class TestPersistentStoragePolicySatisfier {
       DFSTestUtil.waitExpectedStorageType(
           parentFileName, StorageType.ARCHIVE, 3, timeout, fs);
       DFSTestUtil.waitExpectedStorageType(
-          childFileName, StorageType.DEFAULT, 3, timeout, fs);
+          childFileName, StorageType.ARCHIVE, 3, timeout, fs);
     } finally {
       clusterShutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
index aca9a73..d36b147 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
@@ -32,7 +32,6 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Supplier;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -64,7 +63,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -72,7 +70,6 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import org.junit.rules.Timeout;
 import org.mockito.internal.util.reflection.Whitebox;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e64a4280/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionHandler.java
index e2035ed..3481b42 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionHandler.java
@@ -75,6 +75,10 @@ public class TestReencryptionHandler {
         CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
     Mockito.when(ezm.getProvider()).thenReturn(
         KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp));
+    FSDirectory fsd = Mockito.mock(FSDirectory.class);
+    FSNamesystem fns = Mockito.mock(FSNamesystem.class);
+    Mockito.when(fsd.getFSNamesystem()).thenReturn(fns);
+    Mockito.when(ezm.getFSDirectory()).thenReturn(fsd);
     return new ReencryptionHandler(ezm, conf);
   }
 
@@ -99,7 +103,7 @@ public class TestReencryptionHandler {
     Whitebox.setInternalState(rh, "throttleTimerLocked", mockLocked);
     Whitebox.setInternalState(rh, "taskQueue", queue);
     final StopWatch sw = new StopWatch().start();
-    rh.throttle();
+    rh.getTraverser().throttle();
     sw.stop();
     assertTrue("should have throttled for at least 8 second",
         sw.now(TimeUnit.MILLISECONDS) > 8000);
@@ -130,7 +134,7 @@ public class TestReencryptionHandler {
         submissions = new HashMap<>();
     Whitebox.setInternalState(rh, "submissions", submissions);
     StopWatch sw = new StopWatch().start();
-    rh.throttle();
+    rh.getTraverser().throttle();
     sw.stop();
     assertTrue("should not have throttled",
         sw.now(TimeUnit.MILLISECONDS) < 1000);
@@ -189,7 +193,7 @@ public class TestReencryptionHandler {
     Whitebox.setInternalState(rh, "submissions", submissions);
     final StopWatch sw = new StopWatch().start();
     removeTaskThread.start();
-    rh.throttle();
+    rh.getTraverser().throttle();
     sw.stop();
     LOG.info("Throttle completed, consumed {}", sw.now(TimeUnit.MILLISECONDS));
     assertTrue("should have throttled for at least 3 second",


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


[32/50] [abbrv] hadoop git commit: HDFS-11966. [SPS] Correct the log in BlockStorageMovementAttemptedItems#blockStorageMovementResultCheck. Contributed by Surendra Singh Lilhore.

Posted by ra...@apache.org.
HDFS-11966. [SPS] Correct the log in BlockStorageMovementAttemptedItems#blockStorageMovementResultCheck. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: 5262c6502bf1a1402c99835c7cd130ad27540f3b
Parents: 121dc7a
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Sun Jun 18 11:00:28 2017 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:30:08 2017 +0530

----------------------------------------------------------------------
 .../BlockStorageMovementAttemptedItems.java     | 39 ++++++++++----------
 1 file changed, 20 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5262c650/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 bf7859c..6048986 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
@@ -296,19 +296,17 @@ public class BlockStorageMovementAttemptedItems {
             .next();
         synchronized (storageMovementAttemptedItems) {
           Status status = storageMovementAttemptedResult.getStatus();
+          long trackId = storageMovementAttemptedResult.getTrackId();
           ItemInfo itemInfo;
           switch (status) {
           case FAILURE:
-            blockStorageMovementNeeded
-                .add(storageMovementAttemptedResult.getTrackId());
+            blockStorageMovementNeeded.add(trackId);
             LOG.warn("Blocks storage movement results for the tracking id: {}"
                 + " is reported from co-ordinating datanode, but result"
-                + " status is FAILURE. So, added for retry",
-                storageMovementAttemptedResult.getTrackId());
+                + " status is FAILURE. So, added for retry", trackId);
             break;
           case SUCCESS:
-            itemInfo = storageMovementAttemptedItems
-                .get(storageMovementAttemptedResult.getTrackId());
+            itemInfo = storageMovementAttemptedItems.get(trackId);
 
             // ItemInfo could be null. One case is, before the blocks movements
             // result arrives the attempted trackID became timed out and then
@@ -318,20 +316,23 @@ public class BlockStorageMovementAttemptedItems {
             // following condition. If all the block locations under the trackID
             // are attempted and failed to find matching target nodes to satisfy
             // storage policy in previous SPS iteration.
-            if (itemInfo != null
-                && !itemInfo.isAllBlockLocsAttemptedToSatisfy()) {
-              blockStorageMovementNeeded
-                  .add(storageMovementAttemptedResult.getTrackId());
-              LOG.warn("Blocks storage movement is SUCCESS for the track id: {}"
-                  + " reported from co-ordinating datanode. But adding trackID"
-                  + " back to retry queue as some of the blocks couldn't find"
-                  + " matching target nodes in previous SPS iteration.",
-                  storageMovementAttemptedResult.getTrackId());
+            String msg = "Blocks storage movement is SUCCESS for the track id: "
+                + trackId + " reported from co-ordinating datanode.";
+            if (itemInfo != null) {
+              if (!itemInfo.isAllBlockLocsAttemptedToSatisfy()) {
+                blockStorageMovementNeeded.add(trackId);
+                LOG.warn("{} But adding trackID back to retry queue as some of"
+                    + " the blocks couldn't find matching target nodes in"
+                    + " previous SPS iteration.", msg);
+              } else {
+                LOG.info(msg);
+                // Remove xattr for the track id.
+                this.sps.postBlkStorageMovementCleanup(
+                    storageMovementAttemptedResult.getTrackId());
+              }
             } else {
-              LOG.info("Blocks storage movement is SUCCESS for the track id: {}"
-                  + " reported from co-ordinating datanode. But the trackID "
-                  + "doesn't exists in storageMovementAttemptedItems list",
-                  storageMovementAttemptedResult.getTrackId());
+              LOG.info("{} But the trackID doesn't exists in "
+                  + "storageMovementAttemptedItems list", msg);
               // Remove xattr for the track id.
               this.sps.postBlkStorageMovementCleanup(
                   storageMovementAttemptedResult.getTrackId());


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


[13/50] [abbrv] hadoop git commit: HDFS-11123. [SPS] Make storage policy satisfier daemon work on/off dynamically. Contributed by Uma Maheswara Rao G

Posted by ra...@apache.org.
HDFS-11123. [SPS] Make storage policy satisfier daemon work on/off dynamically. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: b19b083b8dc0c69995d2c6f283a1345c6f0afce6
Parents: aca6ab6
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Wed Dec 14 17:49:44 2016 +0530
Committer: Rakesh Radhakrishnan <ra...@apache.org>
Committed: Mon Dec 11 11:28:20 2017 +0530

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |  45 +++++++++
 .../BlockStorageMovementAttemptedItems.java     |  24 +++--
 .../namenode/BlockStorageMovementNeeded.java    |   4 +
 .../hdfs/server/namenode/FSDirAttrOp.java       |   8 --
 .../hdfs/server/namenode/FSNamesystem.java      |  35 ++++++-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  26 ++++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   4 +-
 .../server/namenode/StoragePolicySatisfier.java |  45 +++++++--
 .../src/main/resources/hdfs-default.xml         |   7 +-
 .../namenode/TestNameNodeReconfigure.java       | 100 +++++++++++++++++++
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |   2 +-
 11 files changed, 265 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/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 1c2b0a2..74ee4b5 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
@@ -4948,7 +4948,52 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  /**
+   * Gets the storage policy satisfier instance.
+   *
+   * @return sps
+   */
   public StoragePolicySatisfier getStoragePolicySatisfier() {
     return sps;
   }
+
+  /**
+   * Activate the storage policy satisfier by starting its service.
+   */
+  public void activateSPS() {
+    if (sps == null) {
+      LOG.info("Storage policy satisfier is not initialized.");
+      return;
+    } else if (sps.isRunning()) {
+      LOG.info("Storage policy satisfier is already running.");
+      return;
+    }
+    sps.start();
+  }
+
+  /**
+   * Deactivate the storage policy satisfier by stopping its services.
+   */
+  public void deactivateSPS() {
+    if (sps == null) {
+      LOG.info("Storage policy satisfier is not initialized.");
+      return;
+    } else if (!sps.isRunning()) {
+      LOG.info("Storage policy satisfier is already stopped.");
+      return;
+    }
+    sps.stop();
+    // TODO: add command to DNs for stop in-progress processing SPS commands?
+    // to avoid confusions in cluster, I think sending commands from centralized
+    // place would be better to drop pending queues at DN. Anyway in progress
+    // work will be finished in a while, but this command can void starting
+    // fresh movements at DN.
+  }
+
+  /**
+   * @return True if storage policy satisfier running.
+   */
+  public boolean isStoragePolicySatisfierRunning() {
+    return sps == null ? false : sps.isRunning();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/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 5457dc2..bb26082 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
@@ -49,7 +49,7 @@ public class BlockStorageMovementAttemptedItems {
   // processing and sent to DNs.
   private final Map<Long, Long> storageMovementAttemptedItems;
   private final List<BlocksStorageMovementResult> storageMovementAttemptedResults;
-  private volatile boolean spsRunning = true;
+  private volatile boolean monitorRunning = true;
   private Daemon timerThread = null;
   //
   // It might take anywhere between 30 to 60 minutes before
@@ -109,7 +109,8 @@ public class BlockStorageMovementAttemptedItems {
   /**
    * Starts the monitor thread.
    */
-  void start() {
+  public synchronized void start() {
+    monitorRunning = true;
     timerThread = new Daemon(new BlocksStorageMovementAttemptResultMonitor());
     timerThread.setName("BlocksStorageMovementAttemptResultMonitor");
     timerThread.start();
@@ -118,8 +119,14 @@ public class BlockStorageMovementAttemptedItems {
   /**
    * Stops the monitor thread.
    */
-  public void stop() {
-    spsRunning = false;
+  public synchronized void stop() {
+    monitorRunning = false;
+    timerThread.interrupt();
+    try {
+      timerThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+    this.clearQueues();
   }
 
   /**
@@ -129,13 +136,13 @@ public class BlockStorageMovementAttemptedItems {
   private class BlocksStorageMovementAttemptResultMonitor implements Runnable {
     @Override
     public void run() {
-      while (spsRunning) {
+      while (monitorRunning) {
         try {
           blockStorageMovementResultCheck();
           blocksStorageMovementUnReportedItemsCheck();
           Thread.sleep(checkTimeout);
         } catch (InterruptedException ie) {
-          LOG.debug("BlocksStorageMovementAttemptResultMonitor thread "
+          LOG.info("BlocksStorageMovementAttemptResultMonitor thread "
               + "is interrupted.", ie);
         }
       }
@@ -222,4 +229,9 @@ public class BlockStorageMovementAttemptedItems {
   public int getAttemptedItemsCount() {
     return storageMovementAttemptedItems.size();
   }
+
+  public void clearQueues() {
+    storageMovementAttemptedResults.clear();
+    storageMovementAttemptedItems.clear();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
index c916672..3241e6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
@@ -50,4 +50,8 @@ public class BlockStorageMovementNeeded {
   public synchronized Long get() {
     return storageMovementNeeded.poll();
   }
+
+  public synchronized void clearAll() {
+    storageMovementNeeded.clear();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 31005d5..7743f0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -200,14 +200,6 @@ public class FSDirAttrOp {
   static void satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
       String src) throws IOException {
 
-    // make sure storage policy is enabled, otherwise
-    // there is no need to satisfy storage policy.
-    if (!fsd.isStoragePolicyEnabled()) {
-      throw new IOException(String.format(
-          "Failed to satisfy storage policy since %s is set to false.",
-          DFS_STORAGE_POLICY_ENABLED_KEY));
-    }
-
     FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/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 ec7aafc..6d4e65f 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
@@ -89,7 +89,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT;
+
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
@@ -2166,6 +2168,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot satisfy storage policy for " + src);
+      // make sure storage policy is enabled, otherwise
+      // there is no need to satisfy storage policy.
+      if (!dir.isStoragePolicyEnabled()) {
+        throw new IOException(String.format(
+            "Failed to satisfy storage policy since %s is set to false.",
+            DFS_STORAGE_POLICY_ENABLED_KEY));
+      }
+
+      if (blockManager.getStoragePolicySatisfier() == null
+          || !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 "
+                + "or use Mover tool.");
+      }
       // TODO: need to update editlog for persistence.
       FSDirAttrOp.satisfyStoragePolicy(dir, blockManager, src);
     } finally {
@@ -3790,11 +3808,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         blockReportLeaseId =  blockManager.requestBlockReportLeaseId(nodeReg);
       }
 
-      // TODO: Handle blocks movement results send by the coordinator datanode.
-      // This has to be revisited as part of HDFS-11029.
-      if (blockManager.getStoragePolicySatisfier() != null) {
-        blockManager.getStoragePolicySatisfier()
-            .handleBlocksStorageMovementResults(blksMovementResults);
+      // Handle blocks movement results sent by the coordinator datanode.
+      StoragePolicySatisfier sps = blockManager.getStoragePolicySatisfier();
+      if (sps != null) {
+        if (!sps.isRunning()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Storage policy satisfier is not running. So, ignoring block "
+                    + "storage movement results sent by co-ordinator datanode");
+          }
+        } else {
+          sps.handleBlocksStorageMovementResults(blksMovementResults);
+        }
       }
 
       //create ha status

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/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 32b873b..4f1423a 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
@@ -158,6 +158,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.util.ExitUtil.terminate;
 import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE;
@@ -289,7 +290,8 @@ public class NameNode extends ReconfigurableBase implements
           DFS_HEARTBEAT_INTERVAL_KEY,
           DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
           FS_PROTECTED_DIRECTORIES,
-          HADOOP_CALLER_CONTEXT_ENABLED_KEY));
+          HADOOP_CALLER_CONTEXT_ENABLED_KEY,
+          DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY));
 
   private static final String USAGE = "Usage: hdfs namenode ["
       + StartupOption.BACKUP.getName() + "] | \n\t["
@@ -2050,6 +2052,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 {
       throw new ReconfigurationException(property, newVal, getConf().get(
           property));
@@ -2133,6 +2137,26 @@ public class NameNode extends ReconfigurableBase implements
     return Boolean.toString(clientBackoffEnabled);
   }
 
+  String reconfigureSPSActivate(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, "
+                  + "we must pass true/false only"));
+    }
+
+    boolean activateSPS = Boolean.parseBoolean(newVal);
+    if (activateSPS) {
+      namesystem.getBlockManager().activateSPS();
+    } else {
+      namesystem.getBlockManager().deactivateSPS();
+    }
+    return newVal;
+  }
+
   @Override  // ReconfigurableBase
   protected Configuration getNewConf() {
     return new HdfsConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index a7cf273..5c49e1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -2473,8 +2473,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (nn.isStandbyState()) {
       throw new StandbyException("Not supported by Standby Namenode.");
     }
-    StoragePolicySatisfier sps = namesystem.getBlockManager()
-        .getStoragePolicySatisfier();
-    return sps != null && sps.isRunning();
+    return namesystem.getBlockManager().isStoragePolicySatisfierRunning();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/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 cc2ca7d..56a531f 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
@@ -91,7 +91,9 @@ public class StoragePolicySatisfier implements Runnable {
    * Start storage policy satisfier demon thread. Also start block storage
    * movements monitor for retry the attempts if needed.
    */
-  public void start() {
+  public synchronized void start() {
+    isRunning = true;
+    LOG.info("Starting StoragePolicySatisfier.");
     storagePolicySatisfierThread = new Daemon(this);
     storagePolicySatisfierThread.setName("StoragePolicySatisfier");
     storagePolicySatisfierThread.start();
@@ -101,8 +103,9 @@ public class StoragePolicySatisfier implements Runnable {
   /**
    * Stop storage policy satisfier demon thread.
    */
-  public void stop() {
+  public synchronized void stop() {
     isRunning = false;
+    LOG.info("Stopping StoragePolicySatisfier.");
     if (storagePolicySatisfierThread == null) {
       return;
     }
@@ -112,6 +115,7 @@ public class StoragePolicySatisfier implements Runnable {
     } catch (InterruptedException ie) {
     }
     this.storageMovementsMonitor.stop();
+    this.clearQueues();
   }
 
   /**
@@ -141,14 +145,20 @@ public class StoragePolicySatisfier implements Runnable {
 
   @Override
   public void run() {
-    isRunning = !checkIfMoverRunning();
-    if (!isRunning) {
-      LOG.error("StoragePolicySatisfier thread stopped "
-          + "as Mover ID file " + HdfsServerConstants.MOVER_ID_PATH.toString()
-          + " exists");
-      return;
+    boolean isMoverRunning = !checkIfMoverRunning();
+    synchronized (this) {
+      isRunning = isMoverRunning;
+      if (!isRunning) {
+        // Stopping monitor thread and clearing queues as well
+        this.clearQueues();
+        this.storageMovementsMonitor.stop();
+        LOG.error(
+            "Stopping StoragePolicySatisfier thread " + "as Mover ID file "
+                + HdfsServerConstants.MOVER_ID_PATH.toString() + " exists");
+        return;
+      }
     }
-    while (namesystem.isRunning()) {
+    while (namesystem.isRunning() && isRunning) {
       try {
         Long blockCollectionID = storageMovementNeeded.get();
         if (blockCollectionID != null) {
@@ -159,7 +169,12 @@ public class StoragePolicySatisfier implements Runnable {
         // we want to check block movements.
         Thread.sleep(3000);
       } catch (Throwable t) {
-        isRunning = false;
+        synchronized (this) {
+          isRunning = false;
+          // Stopping monitor thread and clearing queues as well
+          this.clearQueues();
+          this.storageMovementsMonitor.stop();
+        }
         if (!namesystem.isRunning()) {
           LOG.info("Stopping StoragePolicySatisfier.");
           if (!(t instanceof InterruptedException)) {
@@ -488,4 +503,14 @@ public class StoragePolicySatisfier implements Runnable {
   BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
     return storageMovementsMonitor;
   }
+
+  /**
+   * Clear the queues from to be storage movement needed lists and items tracked
+   * in storage movement monitor.
+   */
+  public void clearQueues() {
+    LOG.warn("Clearing all the queues from StoragePolicySatisfier. So, "
+        + "user requests on satisfying block storages would be discarded.");
+    storageMovementNeeded.clearAll();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/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 99ac219..71c5e3a 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
@@ -4347,8 +4347,13 @@
   <name>dfs.storage.policy.satisfier.activate</name>
   <value>true</value>
   <description>
-    If true, activate StoragePolicySatisfier.
+    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.
+    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
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/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 c0de63a..3e7f2e3 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
@@ -30,9 +30,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_CALLER_CONTEXT_ENABLED_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_CALLER_CONTEXT_ENABLED_DEFAULT;
@@ -40,6 +44,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_BLOCK_INVALIDATE_LIMIT_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE_DEFAULT;
 
@@ -216,6 +222,100 @@ public class TestNameNodeReconfigure {
         datanodeManager.getHeartbeatRecheckInterval());
   }
 
+  /**
+   * Tests activate/deactivate Storage Policy Satisfier dynamically.
+   */
+  @Test(timeout = 30000)
+  public void testReconfigureStoragePolicySatisfierActivated()
+      throws ReconfigurationException {
+    final NameNode nameNode = cluster.getNameNode();
+
+    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        true);
+    // try invalid values
+    try {
+      nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+          "text");
+      fail("ReconfigurationException expected");
+    } catch (ReconfigurationException e) {
+      GenericTestUtils.assertExceptionContains(
+          "For activating or deactivating storage policy satisfier, "
+              + "we must pass true/false only",
+          e.getCause());
+    }
+
+    // enable SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        "true");
+
+    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        true);
+
+    // disable SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        "false");
+    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        false);
+
+    // revert to default
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        "true");
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+        true, nameNode.getNamesystem().getBlockManager()
+            .isStoragePolicySatisfierRunning());
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+        true, nameNode.getConf()
+            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false));
+  }
+
+  /**
+   * Test to satisfy storage policy after deactivating storage policy satisfier.
+   */
+  @Test(timeout = 30000)
+  public void testSatisfyStoragePolicyAfterSatisfierDeactivated()
+      throws ReconfigurationException, IOException {
+    final NameNode nameNode = cluster.getNameNode();
+
+    // deactivate SPS
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        "false");
+    verifySPSActivated(nameNode, DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        false);
+
+    Path filePath = new Path("/testSPS");
+    DistributedFileSystem fileSystem = cluster.getFileSystem();
+    fileSystem.create(filePath);
+    fileSystem.setStoragePolicy(filePath, "COLD");
+    try {
+      fileSystem.satisfyStoragePolicy(filePath);
+      fail("Expected to fail, as storage policy feature has deactivated.");
+    } 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 "
+              + "or use Mover tool.", e);
+    }
+
+    // revert to default
+    nameNode.reconfigureProperty(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY,
+        "true");
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+        true, nameNode.getNamesystem().getBlockManager()
+            .isStoragePolicySatisfierRunning());
+    assertEquals(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY + " has wrong value",
+        true, nameNode.getConf()
+            .getBoolean(DFS_STORAGE_POLICY_SATISFIER_ACTIVATE_KEY, false));
+  }
+
+  void verifySPSActivated(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));
+  }
+
   @Test
   public void testBlockInvalidateLimitAfterReconfigured()
       throws ReconfigurationException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b19b083b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
index 6a01de2..e8f9c1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
@@ -372,7 +372,7 @@ public class TestDFSAdmin {
     final List<String> outs = Lists.newArrayList();
     final List<String> errs = Lists.newArrayList();
     getReconfigurableProperties("namenode", address, outs, errs);
-    assertEquals(6, outs.size());
+    assertEquals(7, outs.size());
     assertEquals(DFS_HEARTBEAT_INTERVAL_KEY, outs.get(1));
     assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(2));
     assertEquals(errs.size(), 0);


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